diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml
index 8cca0e7f82ca9..239cdb609f949 100644
--- a/.github/workflows/master.yml
+++ b/.github/workflows/master.yml
@@ -31,7 +31,7 @@ jobs:
run: |
export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN"
export MAVEN_CLI_OPTS="--no-transfer-progress"
- ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} install
+ ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -Pspark-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} install
lint:
diff --git a/.gitignore b/.gitignore
index ae20c85ebe351..06f47161b5f3f 100644
--- a/.gitignore
+++ b/.gitignore
@@ -85,6 +85,7 @@ TempStatsStore/
metastore/
metastore_db/
sql/hive-thriftserver/test_warehouses
+sql/thriftserver/test_warehouse
warehouse/
spark-warehouse/
diff --git a/assembly/pom.xml b/assembly/pom.xml
index ef916fb99a04c..33e1ca6ac4644 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -183,6 +183,16 @@
+
+ spark-thriftserver
+
+
+ org.apache.spark
+ spark-thriftserver_${scala.binary.version}
+ ${project.version}
+
+
+
spark-ganglia-lgpl
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 8a03af5e38c9b..131dbc38ae6fb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -1041,7 +1041,8 @@ object SparkSubmit extends CommandLineUtils with Logging {
* Return whether the given main class represents a thrift server.
*/
private def isThriftServer(mainClass: String): Boolean = {
- mainClass == "org.apache.spark.sql.hive.thriftserver.HiveThriftServer2"
+ mainClass == "org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" ||
+ mainClass == "org.apache.spark.sql.thriftserver.server.SparkThriftServer"
}
/**
diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh
index f1069d4490b5a..3b8189814fb77 100755
--- a/dev/create-release/release-build.sh
+++ b/dev/create-release/release-build.sh
@@ -136,7 +136,7 @@ if [[ $SPARK_VERSION > "2.4" ]]; then
fi
# Hive-specific profiles for some builds
-HIVE_PROFILES="-Phive -Phive-thriftserver"
+HIVE_PROFILES="-Phive -Phive-thriftserver -Pspark-thriftserver"
# Profiles for publishing snapshots and release to Maven Central
PUBLISH_PROFILES="$BASE_PROFILES $HIVE_PROFILES -Pspark-ganglia-lgpl -Pkinesis-asl"
# Profiles for building binary releases
diff --git a/dev/lint-java b/dev/lint-java
index 1f0b0c8379ed0..f2bf4ce5d2a78 100755
--- a/dev/lint-java
+++ b/dev/lint-java
@@ -20,7 +20,7 @@
SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )"
SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)"
-ERRORS=$($SCRIPT_DIR/../build/mvn -Pkinesis-asl -Pmesos -Pkubernetes -Pyarn -Phive -Phive-thriftserver checkstyle:check | grep ERROR)
+ERRORS=$($SCRIPT_DIR/../build/mvn -Pkinesis-asl -Pmesos -Pkubernetes -Pyarn -Phive -Phive-thriftserver -Pspark-thriftserver checkstyle:check | grep ERROR)
if test ! -z "$ERRORS"; then
echo -e "Checkstyle checks failed at following occurrences:\n$ERRORS"
diff --git a/dev/mima b/dev/mima
index f324c5c00a45c..9712d449c275c 100755
--- a/dev/mima
+++ b/dev/mima
@@ -24,7 +24,7 @@ set -e
FWDIR="$(cd "`dirname "$0"`"/..; pwd)"
cd "$FWDIR"
-SPARK_PROFILES=${1:-"-Pmesos -Pkubernetes -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive"}
+SPARK_PROFILES=${1:-"-Pmesos -Pkubernetes -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Pspark-thriftserver -Phive"}
TOOLS_CLASSPATH="$(build/sbt -DcopyDependencies=false "export tools/fullClasspath" | tail -n1)"
OLD_DEPS_CLASSPATH="$(build/sbt -DcopyDependencies=false $SPARK_PROFILES "export oldDeps/fullClasspath" | tail -n1)"
diff --git a/dev/run-tests.py b/dev/run-tests.py
index ea515708124db..652c6dee3fd9e 100755
--- a/dev/run-tests.py
+++ b/dev/run-tests.py
@@ -113,7 +113,7 @@ def determine_modules_to_test(changed_modules):
>>> x = [x.name for x in determine_modules_to_test([modules.sql])]
>>> x # doctest: +NORMALIZE_WHITESPACE
['sql', 'avro', 'hive', 'mllib', 'sql-kafka-0-10', 'examples', 'hive-thriftserver',
- 'pyspark-sql', 'repl', 'sparkr', 'pyspark-mllib', 'pyspark-ml']
+ 'spark-thriftserver', 'pyspark-sql', 'repl', 'sparkr', 'pyspark-mllib', 'pyspark-ml']
"""
modules_to_test = set()
for module in changed_modules:
diff --git a/dev/sbt-checkstyle b/dev/sbt-checkstyle
index 415155fd1bb19..35cd1e71b6b60 100755
--- a/dev/sbt-checkstyle
+++ b/dev/sbt-checkstyle
@@ -17,7 +17,7 @@
# limitations under the License.
#
-SPARK_PROFILES=${1:-"-Pkinesis-asl -Pmesos -Pkubernetes -Pyarn -Phive -Phive-thriftserver"}
+SPARK_PROFILES=${1:-"-Pkinesis-asl -Pmesos -Pkubernetes -Pyarn -Phive -Phive-thriftserver -Pspark-thriftserver"}
# NOTE: echo "q" is needed because SBT prompts the user for input on encountering a build file
# with failure (either resolution or compilation); the "q" makes SBT quit.
diff --git a/dev/scalastyle b/dev/scalastyle
index 212ef900eb9b4..e86ff5c1a507f 100755
--- a/dev/scalastyle
+++ b/dev/scalastyle
@@ -17,7 +17,7 @@
# limitations under the License.
#
-SPARK_PROFILES=${1:-"-Pmesos -Pkubernetes -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive"}
+SPARK_PROFILES=${1:-"-Pmesos -Pkubernetes -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Pspark-thriftserver -Phive"}
# NOTE: echo "q" is needed because SBT prompts the user for input on encountering a build file
# with failure (either resolution or compilation); the "q" makes SBT quit.
diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh
index cc0292e9c2ea5..69ed7e2000ba2 100755
--- a/dev/test-dependencies.sh
+++ b/dev/test-dependencies.sh
@@ -29,7 +29,7 @@ export LC_ALL=C
# TODO: This would be much nicer to do in SBT, once SBT supports Maven-style resolution.
# NOTE: These should match those in the release publishing script
-HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkubernetes -Pyarn -Phive"
+HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pspark-thriftserver -Pmesos -Pkubernetes -Pyarn -Phive"
MVN="build/mvn"
HADOOP_PROFILES=(
hadoop-2.7
diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java
index 3ae4633c79b04..bcdae10920a05 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java
@@ -167,6 +167,7 @@ List buildClassPath(String appClassPath) throws IOException {
"sql/core",
"sql/hive",
"sql/hive-thriftserver",
+ "sql/thriftserver",
"streaming"
);
if (prependClasses) {
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
index 383c3f60a595b..b5106ab5ad114 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
@@ -86,6 +86,8 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder {
SparkLauncher.NO_RESOURCE);
specialClasses.put("org.apache.spark.sql.hive.thriftserver.HiveThriftServer2",
SparkLauncher.NO_RESOURCE);
+ specialClasses.put("org.apache.spark.sql.thriftserver.server.SparkThriftServer",
+ SparkLauncher.NO_RESOURCE);
}
final List userArgs;
@@ -398,7 +400,8 @@ boolean isClientMode(Map userProps) {
*/
private boolean isThriftServer(String mainClass) {
return (mainClass != null &&
- mainClass.equals("org.apache.spark.sql.hive.thriftserver.HiveThriftServer2"));
+ (mainClass.equals("org.apache.spark.sql.hive.thriftserver.HiveThriftServer2")
+ || mainClass.equals("org.apache.spark.sql.thriftserver.server.SparkThriftServer")));
}
private List findExamplesJars() {
diff --git a/pom.xml b/pom.xml
index 8f86ae3b6dfbb..0643a0ba5cbcb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -2978,6 +2978,13 @@
+
+ spark-thriftserver
+
+ sql/thriftserver
+
+
+
hadoop-cloud
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 91d3a75849b0c..0d728e057ba5c 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -42,8 +42,8 @@ object BuildCommons {
private val buildLocation = file(".").getAbsoluteFile.getParentFile
- val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer, tokenProviderKafka010, sqlKafka010, avro) = Seq(
- "catalyst", "sql", "hive", "hive-thriftserver", "token-provider-kafka-0-10", "sql-kafka-0-10", "avro"
+ val sqlProjects@Seq(catalyst, sql, hive, hiveThriftServer, sparkThriftServer, tokenProviderKafka010, sqlKafka010, avro) = Seq(
+ "catalyst", "sql", "hive", "hive-thriftserver", "thriftserver", "token-provider-kafka-0-10", "sql-kafka-0-10", "avro"
).map(ProjectRef(buildLocation, _))
val streamingProjects@Seq(streaming, streamingKafka010) =
@@ -335,7 +335,7 @@ object SparkBuild extends PomBuild {
val mimaProjects = allProjects.filterNot { x =>
Seq(
- spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn,
+ spark, hive, hiveThriftServer, sparkThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn,
unsafe, tags, tokenProviderKafka010, sqlKafka010, kvstore, avro, graph, graphApi, cypher
).contains(x)
}
diff --git a/sbin/start-spark-thriftserver.sh b/sbin/start-spark-thriftserver.sh
new file mode 100755
index 0000000000000..5983326053db0
--- /dev/null
+++ b/sbin/start-spark-thriftserver.sh
@@ -0,0 +1,60 @@
+#!/usr/bin/env bash
+
+#
+# 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.
+#
+
+#
+# Shell script for starting the Spark SQL Thrift server
+
+# Enter posix mode for bash
+set -o posix
+
+if [ -z "${SPARK_HOME}" ]; then
+ export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)"
+fi
+
+# NOTE: This exact class name is matched downstream by SparkSubmit.
+# Any changes need to be reflected there.
+CLASS="org.apache.spark.sql.thriftserver.server.SparkThriftServer"
+
+function usage {
+ echo "Usage: ./sbin/start-spark-thriftserver [options] [thrift server options]"
+ pattern="usage"
+ pattern+="\|Spark assembly has been built with Hive"
+ pattern+="\|NOTE: SPARK_PREPEND_CLASSES is set"
+ pattern+="\|Spark Command: "
+ pattern+="\|======="
+ pattern+="\|--help"
+ pattern+="\|Using Spark's default log4j profile:"
+ pattern+="\|^log4j:"
+ pattern+="\|Started daemon with process name"
+ pattern+="\|Registered signal handler for"
+
+ "${SPARK_HOME}"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2
+ echo
+ echo "Thrift server options:"
+ "${SPARK_HOME}"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2
+}
+
+if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then
+ usage
+ exit 1
+fi
+
+export SUBMIT_USAGE_FUNCTION=usage
+
+exec "${SPARK_HOME}"/sbin/spark-daemon.sh submit $CLASS 1 --name "Thrift JDBC/ODBC Server" "$@"
diff --git a/sbin/stop-spark-thriftserver.sh b/sbin/stop-spark-thriftserver.sh
new file mode 100755
index 0000000000000..e02e47d42aa6c
--- /dev/null
+++ b/sbin/stop-spark-thriftserver.sh
@@ -0,0 +1,26 @@
+#!/usr/bin/env bash
+
+#
+# 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.
+#
+
+# Stops the thrift server on the machine this script is executed on.
+
+if [ -z "${SPARK_HOME}" ]; then
+ export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)"
+fi
+
+"${SPARK_HOME}/sbin"/spark-daemon.sh stop org.apache.spark.sql.thriftserver.server.SparkThriftServer 1
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
index 9c4b8a5819a33..201175d337c5a 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala
@@ -307,7 +307,7 @@ private[spark] object HiveUtils extends Logging {
* The version of the Hive client that is used here must match the metastore that is configured
* in the hive-site.xml file.
*/
- protected[hive] def newClientForMetadata(
+ def newClientForMetadata(
conf: SparkConf,
hadoopConf: Configuration): HiveClient = {
val configurations = formatTimeVarsForHiveClient(hadoopConf)
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala
index 5fceab3105e83..fd85cadc4f242 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala
@@ -648,7 +648,7 @@ private[sql] class TestHiveSessionStateBuilder(
override protected def newBuilder: NewBuilder = new TestHiveSessionStateBuilder(_, _)
}
-private[hive] object HiveTestJars {
+object HiveTestJars {
private val repository = SQLConf.ADDITIONAL_REMOTE_REPOSITORIES.defaultValueString.split(",")(0)
private val hiveTestJarsDir = Utils.createTempDir()
diff --git a/sql/thriftserver/README.md b/sql/thriftserver/README.md
new file mode 100644
index 0000000000000..91a3487d23dc9
--- /dev/null
+++ b/sql/thriftserver/README.md
@@ -0,0 +1,5 @@
+Thrift commands to generate files from TCLIService.thrift :
+--------------------
+Please use Thrift 0.9.3 available from https://www.apache.org/dist/thrift/0.9.3
+
+`thrift --gen java:beans,hashcode,generated_annotations=undated -o src/gen/thrift if/TCLIService.thrift`
diff --git a/sql/thriftserver/if/TCLIService.thrift b/sql/thriftserver/if/TCLIService.thrift
new file mode 100644
index 0000000000000..8899cf2aff711
--- /dev/null
+++ b/sql/thriftserver/if/TCLIService.thrift
@@ -0,0 +1,1297 @@
+// 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.
+
+// Coding Conventions for this file:
+//
+// Structs/Enums/Unions
+// * Struct, Enum, and Union names begin with a "T",
+// and use a capital letter for each new word, with no underscores.
+// * All fields should be declared as either optional or required.
+//
+// Functions
+// * Function names start with a capital letter and have a capital letter for
+// each new word, with no underscores.
+// * Each function should take exactly one parameter, named TFunctionNameReq,
+// and should return either void or TFunctionNameResp. This convention allows
+// incremental updates.
+//
+// Services
+// * Service names begin with the letter "T", use a capital letter for each
+// new word (with no underscores), and end with the word "Service".
+
+namespace java org.apache.spark.sql.thriftserver.cli.thrift
+namespace cpp apache.spark.sql.thriftserver.cli.thrift
+
+// List of protocol versions. A new token should be
+// added to the end of this list every time a change is made.
+enum TProtocolVersion {
+ HIVE_CLI_SERVICE_PROTOCOL_V1,
+
+ // V2 adds support for asynchronous execution
+ HIVE_CLI_SERVICE_PROTOCOL_V2
+
+ // V3 add varchar type, primitive type qualifiers
+ HIVE_CLI_SERVICE_PROTOCOL_V3
+
+ // V4 add decimal precision/scale, char type
+ HIVE_CLI_SERVICE_PROTOCOL_V4
+
+ // V5 adds error details when GetOperationStatus returns in error state
+ HIVE_CLI_SERVICE_PROTOCOL_V5
+
+ // V6 uses binary type for binary payload (was string) and uses columnar result set
+ HIVE_CLI_SERVICE_PROTOCOL_V6
+
+ // V7 adds support for delegation token based connection
+ HIVE_CLI_SERVICE_PROTOCOL_V7
+
+ // V8 adds support for interval types
+ HIVE_CLI_SERVICE_PROTOCOL_V8
+
+ // V9 adds support for serializing ResultSets in SerDe
+ HIVE_CLI_SERVICE_PROTOCOL_V9
+
+ // V10 adds support for in place updates via GetOperationStatus
+ HIVE_CLI_SERVICE_PROTOCOL_V10
+
+ // V11 adds timestamp with local time zone type
+ HIVE_CLI_SERVICE_PROTOCOL_V11
+}
+
+enum TTypeId {
+ BOOLEAN_TYPE,
+ TINYINT_TYPE,
+ SMALLINT_TYPE,
+ INT_TYPE,
+ BIGINT_TYPE,
+ FLOAT_TYPE,
+ DOUBLE_TYPE,
+ STRING_TYPE,
+ TIMESTAMP_TYPE,
+ BINARY_TYPE,
+ ARRAY_TYPE,
+ MAP_TYPE,
+ STRUCT_TYPE,
+ UNION_TYPE,
+ USER_DEFINED_TYPE,
+ DECIMAL_TYPE,
+ NULL_TYPE,
+ DATE_TYPE,
+ VARCHAR_TYPE,
+ CHAR_TYPE,
+ INTERVAL_YEAR_MONTH_TYPE,
+ INTERVAL_DAY_TIME_TYPE,
+ TIMESTAMPLOCALTZ_TYPE
+}
+
+const set PRIMITIVE_TYPES = [
+ TTypeId.BOOLEAN_TYPE,
+ TTypeId.TINYINT_TYPE,
+ TTypeId.SMALLINT_TYPE,
+ TTypeId.INT_TYPE,
+ TTypeId.BIGINT_TYPE,
+ TTypeId.FLOAT_TYPE,
+ TTypeId.DOUBLE_TYPE,
+ TTypeId.STRING_TYPE,
+ TTypeId.TIMESTAMP_TYPE,
+ TTypeId.BINARY_TYPE,
+ TTypeId.DECIMAL_TYPE,
+ TTypeId.NULL_TYPE,
+ TTypeId.DATE_TYPE,
+ TTypeId.VARCHAR_TYPE,
+ TTypeId.CHAR_TYPE,
+ TTypeId.INTERVAL_YEAR_MONTH_TYPE,
+ TTypeId.INTERVAL_DAY_TIME_TYPE,
+ TTypeId.TIMESTAMPLOCALTZ_TYPE
+]
+
+const set COMPLEX_TYPES = [
+ TTypeId.ARRAY_TYPE
+ TTypeId.MAP_TYPE
+ TTypeId.STRUCT_TYPE
+ TTypeId.UNION_TYPE
+ TTypeId.USER_DEFINED_TYPE
+]
+
+const set COLLECTION_TYPES = [
+ TTypeId.ARRAY_TYPE
+ TTypeId.MAP_TYPE
+]
+
+const map TYPE_NAMES = {
+ TTypeId.BOOLEAN_TYPE: "BOOLEAN",
+ TTypeId.TINYINT_TYPE: "TINYINT",
+ TTypeId.SMALLINT_TYPE: "SMALLINT",
+ TTypeId.INT_TYPE: "INT",
+ TTypeId.BIGINT_TYPE: "BIGINT",
+ TTypeId.FLOAT_TYPE: "FLOAT",
+ TTypeId.DOUBLE_TYPE: "DOUBLE",
+ TTypeId.STRING_TYPE: "STRING",
+ TTypeId.TIMESTAMP_TYPE: "TIMESTAMP",
+ TTypeId.BINARY_TYPE: "BINARY",
+ TTypeId.ARRAY_TYPE: "ARRAY",
+ TTypeId.MAP_TYPE: "MAP",
+ TTypeId.STRUCT_TYPE: "STRUCT",
+ TTypeId.UNION_TYPE: "UNIONTYPE",
+ TTypeId.DECIMAL_TYPE: "DECIMAL",
+ TTypeId.NULL_TYPE: "NULL"
+ TTypeId.DATE_TYPE: "DATE"
+ TTypeId.VARCHAR_TYPE: "VARCHAR"
+ TTypeId.CHAR_TYPE: "CHAR"
+ TTypeId.INTERVAL_YEAR_MONTH_TYPE: "INTERVAL_YEAR_MONTH"
+ TTypeId.INTERVAL_DAY_TIME_TYPE: "INTERVAL_DAY_TIME"
+ TTypeId.TIMESTAMPLOCALTZ_TYPE: "TIMESTAMP WITH LOCAL TIME ZONE"
+}
+
+// Thrift does not support recursively defined types or forward declarations,
+// which makes it difficult to represent Hive's nested types.
+// To get around these limitations TTypeDesc employs a type list that maps
+// integer "pointers" to TTypeEntry objects. The following examples show
+// how different types are represented using this scheme:
+//
+// "INT":
+// TTypeDesc {
+// types = [
+// TTypeEntry.primitive_entry {
+// type = INT_TYPE
+// }
+// ]
+// }
+//
+// "ARRAY":
+// TTypeDesc {
+// types = [
+// TTypeEntry.array_entry {
+// object_type_ptr = 1
+// },
+// TTypeEntry.primitive_entry {
+// type = INT_TYPE
+// }
+// ]
+// }
+//
+// "MAP":
+// TTypeDesc {
+// types = [
+// TTypeEntry.map_entry {
+// key_type_ptr = 1
+// value_type_ptr = 2
+// },
+// TTypeEntry.primitive_entry {
+// type = INT_TYPE
+// },
+// TTypeEntry.primitive_entry {
+// type = STRING_TYPE
+// }
+// ]
+// }
+
+typedef i32 TTypeEntryPtr
+
+// Valid TTypeQualifiers key names
+const string CHARACTER_MAXIMUM_LENGTH = "characterMaximumLength"
+
+// Type qualifier key name for decimal
+const string PRECISION = "precision"
+const string SCALE = "scale"
+
+union TTypeQualifierValue {
+ 1: optional i32 i32Value
+ 2: optional string stringValue
+}
+
+// Type qualifiers for primitive type.
+struct TTypeQualifiers {
+ 1: required map qualifiers
+}
+
+// Type entry for a primitive type.
+struct TPrimitiveTypeEntry {
+ // The primitive type token. This must satisfy the condition
+ // that type is in the PRIMITIVE_TYPES set.
+ 1: required TTypeId type
+ 2: optional TTypeQualifiers typeQualifiers
+}
+
+// Type entry for an ARRAY type.
+struct TArrayTypeEntry {
+ 1: required TTypeEntryPtr objectTypePtr
+}
+
+// Type entry for a MAP type.
+struct TMapTypeEntry {
+ 1: required TTypeEntryPtr keyTypePtr
+ 2: required TTypeEntryPtr valueTypePtr
+}
+
+// Type entry for a STRUCT type.
+struct TStructTypeEntry {
+ 1: required map nameToTypePtr
+}
+
+// Type entry for a UNIONTYPE type.
+struct TUnionTypeEntry {
+ 1: required map nameToTypePtr
+}
+
+struct TUserDefinedTypeEntry {
+ // The fully qualified name of the class implementing this type.
+ 1: required string typeClassName
+}
+
+// We use a union here since Thrift does not support inheritance.
+union TTypeEntry {
+ 1: TPrimitiveTypeEntry primitiveEntry
+ 2: TArrayTypeEntry arrayEntry
+ 3: TMapTypeEntry mapEntry
+ 4: TStructTypeEntry structEntry
+ 5: TUnionTypeEntry unionEntry
+ 6: TUserDefinedTypeEntry userDefinedTypeEntry
+}
+
+// Type descriptor for columns.
+struct TTypeDesc {
+ // The "top" type is always the first element of the list.
+ // If the top type is an ARRAY, MAP, STRUCT, or UNIONTYPE
+ // type, then subsequent elements represent nested types.
+ 1: required list types
+}
+
+// A result set column descriptor.
+struct TColumnDesc {
+ // The name of the column
+ 1: required string columnName
+
+ // The type descriptor for this column
+ 2: required TTypeDesc typeDesc
+
+ // The ordinal position of this column in the schema
+ 3: required i32 position
+
+ 4: optional string comment
+}
+
+// Metadata used to describe the schema (column names, types, comments)
+// of result sets.
+struct TTableSchema {
+ 1: required list columns
+}
+
+// A Boolean column value.
+struct TBoolValue {
+ // NULL if value is unset.
+ 1: optional bool value
+}
+
+// A Byte column value.
+struct TByteValue {
+ // NULL if value is unset.
+ 1: optional byte value
+}
+
+// A signed, 16 bit column value.
+struct TI16Value {
+ // NULL if value is unset
+ 1: optional i16 value
+}
+
+// A signed, 32 bit column value
+struct TI32Value {
+ // NULL if value is unset
+ 1: optional i32 value
+}
+
+// A signed 64 bit column value
+struct TI64Value {
+ // NULL if value is unset
+ 1: optional i64 value
+}
+
+// A floating point 64 bit column value
+struct TDoubleValue {
+ // NULL if value is unset
+ 1: optional double value
+}
+
+struct TStringValue {
+ // NULL if value is unset
+ 1: optional string value
+}
+
+// A single column value in a result set.
+// Note that Hive's type system is richer than Thrift's,
+// so in some cases we have to map multiple Hive types
+// to the same Thrift type. On the client-side this is
+// disambiguated by looking at the Schema of the
+// result set.
+union TColumnValue {
+ 1: TBoolValue boolVal // BOOLEAN
+ 2: TByteValue byteVal // TINYINT
+ 3: TI16Value i16Val // SMALLINT
+ 4: TI32Value i32Val // INT
+ 5: TI64Value i64Val // BIGINT, TIMESTAMP
+ 6: TDoubleValue doubleVal // FLOAT, DOUBLE
+ 7: TStringValue stringVal // STRING, LIST, MAP, STRUCT, UNIONTYPE, BINARY, DECIMAL, NULL, INTERVAL_YEAR_MONTH, INTERVAL_DAY_TIME
+}
+
+// Represents a row in a rowset.
+struct TRow {
+ 1: required list colVals
+}
+
+struct TBoolColumn {
+ 1: required list values
+ 2: required binary nulls
+}
+
+struct TByteColumn {
+ 1: required list values
+ 2: required binary nulls
+}
+
+struct TI16Column {
+ 1: required list values
+ 2: required binary nulls
+}
+
+struct TI32Column {
+ 1: required list values
+ 2: required binary nulls
+}
+
+struct TI64Column {
+ 1: required list values
+ 2: required binary nulls
+}
+
+struct TDoubleColumn {
+ 1: required list values
+ 2: required binary nulls
+}
+
+struct TStringColumn {
+ 1: required list values
+ 2: required binary nulls
+}
+
+struct TBinaryColumn {
+ 1: required list values
+ 2: required binary nulls
+}
+
+// Note that Hive's type system is richer than Thrift's,
+// so in some cases we have to map multiple Hive types
+// to the same Thrift type. On the client-side this is
+// disambiguated by looking at the Schema of the
+// result set.
+union TColumn {
+ 1: TBoolColumn boolVal // BOOLEAN
+ 2: TByteColumn byteVal // TINYINT
+ 3: TI16Column i16Val // SMALLINT
+ 4: TI32Column i32Val // INT
+ 5: TI64Column i64Val // BIGINT, TIMESTAMP
+ 6: TDoubleColumn doubleVal // FLOAT, DOUBLE
+ 7: TStringColumn stringVal // STRING, LIST, MAP, STRUCT, UNIONTYPE, DECIMAL, NULL
+ 8: TBinaryColumn binaryVal // BINARY
+}
+
+// Represents a rowset
+struct TRowSet {
+ // The starting row offset of this rowset.
+ 1: required i64 startRowOffset
+ 2: required list rows
+ 3: optional list columns
+ 4: optional binary binaryColumns
+ 5: optional i32 columnCount
+}
+
+// The return status code contained in each response.
+enum TStatusCode {
+ SUCCESS_STATUS,
+ SUCCESS_WITH_INFO_STATUS,
+ STILL_EXECUTING_STATUS,
+ ERROR_STATUS,
+ INVALID_HANDLE_STATUS
+}
+
+// The return status of a remote request
+struct TStatus {
+ 1: required TStatusCode statusCode
+
+ // If status is SUCCESS_WITH_INFO, info_msgs may be populated with
+ // additional diagnostic information.
+ 2: optional list infoMessages
+
+ // If status is ERROR, then the following fields may be set
+ 3: optional string sqlState // as defined in the ISO/IEF CLI specification
+ 4: optional i32 errorCode // internal error code
+ 5: optional string errorMessage
+}
+
+// The state of an operation (i.e. a query or other
+// asynchronous operation that generates a result set)
+// on the server.
+enum TOperationState {
+ // The operation has been initialized
+ INITIALIZED_STATE,
+
+ // The operation is running. In this state the result
+ // set is not available.
+ RUNNING_STATE,
+
+ // The operation has completed. When an operation is in
+ // this state its result set may be fetched.
+ FINISHED_STATE,
+
+ // The operation was canceled by a client
+ CANCELED_STATE,
+
+ // The operation was closed by a client
+ CLOSED_STATE,
+
+ // The operation failed due to an error
+ ERROR_STATE,
+
+ // The operation is in an unrecognized state
+ UKNOWN_STATE,
+
+ // The operation is in an pending state
+ PENDING_STATE,
+
+ // The operation is in an timedout state
+ TIMEDOUT_STATE,
+}
+
+// A string identifier. This is interpreted literally.
+typedef string TIdentifier
+
+// A search pattern.
+//
+// Valid search pattern characters:
+// '_': Any single character.
+// '%': Any sequence of zero or more characters.
+// '\': Escape character used to include special characters,
+// e.g. '_', '%', '\'. If a '\' precedes a non-special
+// character it has no special meaning and is interpreted
+// literally.
+typedef string TPattern
+
+
+// A search pattern or identifier. Used as input
+// parameter for many of the catalog functions.
+typedef string TPatternOrIdentifier
+
+struct THandleIdentifier {
+ // 16 byte globally unique identifier
+ // This is the public ID of the handle and
+ // can be used for reporting.
+ 1: required binary guid,
+
+ // 16 byte secret generated by the server
+ // and used to verify that the handle is not
+ // being hijacked by another user.
+ 2: required binary secret,
+}
+
+// Client-side handle to persistent
+// session information on the server-side.
+struct TSessionHandle {
+ 1: required THandleIdentifier sessionId
+}
+
+// The subtype of an OperationHandle.
+enum TOperationType {
+ EXECUTE_STATEMENT,
+ GET_TYPE_INFO,
+ GET_CATALOGS,
+ GET_SCHEMAS,
+ GET_TABLES,
+ GET_TABLE_TYPES,
+ GET_COLUMNS,
+ GET_FUNCTIONS,
+ UNKNOWN,
+}
+
+// Client-side reference to a task running
+// asynchronously on the server.
+struct TOperationHandle {
+ 1: required THandleIdentifier operationId
+ 2: required TOperationType operationType
+
+ // If hasResultSet = TRUE, then this operation
+ // generates a result set that can be fetched.
+ // Note that the result set may be empty.
+ //
+ // If hasResultSet = FALSE, then this operation
+ // does not generate a result set, and calling
+ // GetResultSetMetadata or FetchResults against
+ // this OperationHandle will generate an error.
+ 3: required bool hasResultSet
+
+ // For operations that don't generate result sets,
+ // modifiedRowCount is either:
+ //
+ // 1) The number of rows that were modified by
+ // the DML operation (e.g. number of rows inserted,
+ // number of rows deleted, etc).
+ //
+ // 2) 0 for operations that don't modify or add rows.
+ //
+ // 3) < 0 if the operation is capable of modifiying rows,
+ // but Hive is unable to determine how many rows were
+ // modified. For example, Hive's LOAD DATA command
+ // doesn't generate row count information because
+ // Hive doesn't inspect the data as it is loaded.
+ //
+ // modifiedRowCount is unset if the operation generates
+ // a result set.
+ 4: optional double modifiedRowCount
+}
+
+
+// OpenSession()
+//
+// Open a session (connection) on the server against
+// which operations may be executed.
+struct TOpenSessionReq {
+ // The version of the HiveServer2 protocol that the client is using.
+ 1: required TProtocolVersion client_protocol = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V10
+
+ // Username and password for authentication.
+ // Depending on the authentication scheme being used,
+ // this information may instead be provided by a lower
+ // protocol layer, in which case these fields may be
+ // left unset.
+ 2: optional string username
+ 3: optional string password
+
+ // Configuration overlay which is applied when the session is
+ // first created.
+ 4: optional map configuration
+}
+
+struct TOpenSessionResp {
+ 1: required TStatus status
+
+ // The protocol version that the server is using.
+ 2: required TProtocolVersion serverProtocolVersion = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V10
+
+ // Session Handle
+ 3: optional TSessionHandle sessionHandle
+
+ // The configuration settings for this session.
+ 4: optional map configuration
+}
+
+struct TSetClientInfoReq {
+ 1: required TSessionHandle sessionHandle,
+ 2: optional map configuration
+}
+
+struct TSetClientInfoResp {
+ 1: required TStatus status
+}
+
+
+// CloseSession()
+//
+// Closes the specified session and frees any resources
+// currently allocated to that session. Any open
+// operations in that session will be canceled.
+struct TCloseSessionReq {
+ 1: required TSessionHandle sessionHandle
+}
+
+struct TCloseSessionResp {
+ 1: required TStatus status
+}
+
+
+
+enum TGetInfoType {
+ CLI_MAX_DRIVER_CONNECTIONS = 0,
+ CLI_MAX_CONCURRENT_ACTIVITIES = 1,
+ CLI_DATA_SOURCE_NAME = 2,
+ CLI_FETCH_DIRECTION = 8,
+ CLI_SERVER_NAME = 13,
+ CLI_SEARCH_PATTERN_ESCAPE = 14,
+ CLI_DBMS_NAME = 17,
+ CLI_DBMS_VER = 18,
+ CLI_ACCESSIBLE_TABLES = 19,
+ CLI_ACCESSIBLE_PROCEDURES = 20,
+ CLI_CURSOR_COMMIT_BEHAVIOR = 23,
+ CLI_DATA_SOURCE_READ_ONLY = 25,
+ CLI_DEFAULT_TXN_ISOLATION = 26,
+ CLI_IDENTIFIER_CASE = 28,
+ CLI_IDENTIFIER_QUOTE_CHAR = 29,
+ CLI_MAX_COLUMN_NAME_LEN = 30,
+ CLI_MAX_CURSOR_NAME_LEN = 31,
+ CLI_MAX_SCHEMA_NAME_LEN = 32,
+ CLI_MAX_CATALOG_NAME_LEN = 34,
+ CLI_MAX_TABLE_NAME_LEN = 35,
+ CLI_SCROLL_CONCURRENCY = 43,
+ CLI_TXN_CAPABLE = 46,
+ CLI_USER_NAME = 47,
+ CLI_TXN_ISOLATION_OPTION = 72,
+ CLI_INTEGRITY = 73,
+ CLI_GETDATA_EXTENSIONS = 81,
+ CLI_NULL_COLLATION = 85,
+ CLI_ALTER_TABLE = 86,
+ CLI_ORDER_BY_COLUMNS_IN_SELECT = 90,
+ CLI_SPECIAL_CHARACTERS = 94,
+ CLI_MAX_COLUMNS_IN_GROUP_BY = 97,
+ CLI_MAX_COLUMNS_IN_INDEX = 98,
+ CLI_MAX_COLUMNS_IN_ORDER_BY = 99,
+ CLI_MAX_COLUMNS_IN_SELECT = 100,
+ CLI_MAX_COLUMNS_IN_TABLE = 101,
+ CLI_MAX_INDEX_SIZE = 102,
+ CLI_MAX_ROW_SIZE = 104,
+ CLI_MAX_STATEMENT_LEN = 105,
+ CLI_MAX_TABLES_IN_SELECT = 106,
+ CLI_MAX_USER_NAME_LEN = 107,
+ CLI_OJ_CAPABILITIES = 115,
+
+ CLI_XOPEN_CLI_YEAR = 10000,
+ CLI_CURSOR_SENSITIVITY = 10001,
+ CLI_DESCRIBE_PARAMETER = 10002,
+ CLI_CATALOG_NAME = 10003,
+ CLI_COLLATION_SEQ = 10004,
+ CLI_MAX_IDENTIFIER_LEN = 10005,
+ CLI_ODBC_KEYWORDS = 10006
+}
+
+union TGetInfoValue {
+ 1: string stringValue
+ 2: i16 smallIntValue
+ 3: i32 integerBitmask
+ 4: i32 integerFlag
+ 5: i32 binaryValue
+ 6: i64 lenValue
+}
+
+// GetInfo()
+//
+// This function is based on ODBC's CLIGetInfo() function.
+// The function returns general information about the data source
+// using the same keys as ODBC.
+struct TGetInfoReq {
+ // The sesssion to run this request against
+ 1: required TSessionHandle sessionHandle
+
+ 2: required TGetInfoType infoType
+}
+
+struct TGetInfoResp {
+ 1: required TStatus status
+
+ 2: required TGetInfoValue infoValue
+}
+
+
+// ExecuteStatement()
+//
+// Execute a statement.
+// The returned OperationHandle can be used to check on the
+// status of the statement, and to fetch results once the
+// statement has finished executing.
+struct TExecuteStatementReq {
+ // The session to execute the statement against
+ 1: required TSessionHandle sessionHandle
+
+ // The statement to be executed (DML, DDL, SET, etc)
+ 2: required string statement
+
+ // Configuration properties that are overlayed on top of the
+ // the existing session configuration before this statement
+ // is executed. These properties apply to this statement
+ // only and will not affect the subsequent state of the Session.
+ 3: optional map confOverlay
+
+ // Execute asynchronously when runAsync is true
+ 4: optional bool runAsync = false
+
+ // The number of seconds after which the query will timeout on the server
+ 5: optional i64 queryTimeout = 0
+}
+
+struct TExecuteStatementResp {
+ 1: required TStatus status
+ 2: optional TOperationHandle operationHandle
+}
+
+// GetTypeInfo()
+//
+// Get information about types supported by the HiveServer instance.
+// The information is returned as a result set which can be fetched
+// using the OperationHandle provided in the response.
+//
+// Refer to the documentation for ODBC's CLIGetTypeInfo function for
+// the format of the result set.
+struct TGetTypeInfoReq {
+ // The session to run this request against.
+ 1: required TSessionHandle sessionHandle
+}
+
+struct TGetTypeInfoResp {
+ 1: required TStatus status
+ 2: optional TOperationHandle operationHandle
+}
+
+
+// GetCatalogs()
+//
+// Returns the list of catalogs (databases)
+// Results are ordered by TABLE_CATALOG
+//
+// Resultset columns :
+// col1
+// name: TABLE_CAT
+// type: STRING
+// desc: Catalog name. NULL if not applicable.
+//
+struct TGetCatalogsReq {
+ // Session to run this request against
+ 1: required TSessionHandle sessionHandle
+}
+
+struct TGetCatalogsResp {
+ 1: required TStatus status
+ 2: optional TOperationHandle operationHandle
+}
+
+
+// GetSchemas()
+//
+// Retrieves the schema names available in this database.
+// The results are ordered by TABLE_CATALOG and TABLE_SCHEM.
+// col1
+// name: TABLE_SCHEM
+// type: STRING
+// desc: schema name
+// col2
+// name: TABLE_CATALOG
+// type: STRING
+// desc: catalog name
+struct TGetSchemasReq {
+ // Session to run this request against
+ 1: required TSessionHandle sessionHandle
+
+ // Name of the catalog. Must not contain a search pattern.
+ 2: optional TIdentifier catalogName
+
+ // schema name or pattern
+ 3: optional TPatternOrIdentifier schemaName
+}
+
+struct TGetSchemasResp {
+ 1: required TStatus status
+ 2: optional TOperationHandle operationHandle
+}
+
+
+// GetTables()
+//
+// Returns a list of tables with catalog, schema, and table
+// type information. The information is returned as a result
+// set which can be fetched using the OperationHandle
+// provided in the response.
+// Results are ordered by TABLE_TYPE, TABLE_CAT, TABLE_SCHEM, and TABLE_NAME
+//
+// Result Set Columns:
+//
+// col1
+// name: TABLE_CAT
+// type: STRING
+// desc: Catalog name. NULL if not applicable.
+//
+// col2
+// name: TABLE_SCHEM
+// type: STRING
+// desc: Schema name.
+//
+// col3
+// name: TABLE_NAME
+// type: STRING
+// desc: Table name.
+//
+// col4
+// name: TABLE_TYPE
+// type: STRING
+// desc: The table type, e.g. "TABLE", "VIEW", etc.
+//
+// col5
+// name: REMARKS
+// type: STRING
+// desc: Comments about the table
+//
+struct TGetTablesReq {
+ // Session to run this request against
+ 1: required TSessionHandle sessionHandle
+
+ // Name of the catalog or a search pattern.
+ 2: optional TPatternOrIdentifier catalogName
+
+ // Name of the schema or a search pattern.
+ 3: optional TPatternOrIdentifier schemaName
+
+ // Name of the table or a search pattern.
+ 4: optional TPatternOrIdentifier tableName
+
+ // List of table types to match
+ // e.g. "TABLE", "VIEW", "SYSTEM TABLE", "GLOBAL TEMPORARY",
+ // "LOCAL TEMPORARY", "ALIAS", "SYNONYM", etc.
+ 5: optional list tableTypes
+}
+
+struct TGetTablesResp {
+ 1: required TStatus status
+ 2: optional TOperationHandle operationHandle
+}
+
+
+// GetTableTypes()
+//
+// Returns the table types available in this database.
+// The results are ordered by table type.
+//
+// col1
+// name: TABLE_TYPE
+// type: STRING
+// desc: Table type name.
+struct TGetTableTypesReq {
+ // Session to run this request against
+ 1: required TSessionHandle sessionHandle
+}
+
+struct TGetTableTypesResp {
+ 1: required TStatus status
+ 2: optional TOperationHandle operationHandle
+}
+
+
+// GetColumns()
+//
+// Returns a list of columns in the specified tables.
+// The information is returned as a result set which can be fetched
+// using the OperationHandle provided in the response.
+// Results are ordered by TABLE_CAT, TABLE_SCHEM, TABLE_NAME,
+// and ORDINAL_POSITION.
+//
+// Result Set Columns are the same as those for the ODBC CLIColumns
+// function.
+//
+struct TGetColumnsReq {
+ // Session to run this request against
+ 1: required TSessionHandle sessionHandle
+
+ // Name of the catalog. Must not contain a search pattern.
+ 2: optional TIdentifier catalogName
+
+ // Schema name or search pattern
+ 3: optional TPatternOrIdentifier schemaName
+
+ // Table name or search pattern
+ 4: optional TPatternOrIdentifier tableName
+
+ // Column name or search pattern
+ 5: optional TPatternOrIdentifier columnName
+}
+
+struct TGetColumnsResp {
+ 1: required TStatus status
+ 2: optional TOperationHandle operationHandle
+}
+
+
+// GetFunctions()
+//
+// Returns a list of functions supported by the data source. The
+// behavior of this function matches
+// java.sql.DatabaseMetaData.getFunctions() both in terms of
+// inputs and outputs.
+//
+// Result Set Columns:
+//
+// col1
+// name: FUNCTION_CAT
+// type: STRING
+// desc: Function catalog (may be null)
+//
+// col2
+// name: FUNCTION_SCHEM
+// type: STRING
+// desc: Function schema (may be null)
+//
+// col3
+// name: FUNCTION_NAME
+// type: STRING
+// desc: Function name. This is the name used to invoke the function.
+//
+// col4
+// name: REMARKS
+// type: STRING
+// desc: Explanatory comment on the function.
+//
+// col5
+// name: FUNCTION_TYPE
+// type: SMALLINT
+// desc: Kind of function. One of:
+// * functionResultUnknown - Cannot determine if a return value or a table
+// will be returned.
+// * functionNoTable - Does not a return a table.
+// * functionReturnsTable - Returns a table.
+//
+// col6
+// name: SPECIFIC_NAME
+// type: STRING
+// desc: The name which uniquely identifies this function within its schema.
+// In this case this is the fully qualified class name of the class
+// that implements this function.
+//
+struct TGetFunctionsReq {
+ // Session to run this request against
+ 1: required TSessionHandle sessionHandle
+
+ // A catalog name; must match the catalog name as it is stored in the
+ // database; "" retrieves those without a catalog; null means
+ // that the catalog name should not be used to narrow the search.
+ 2: optional TIdentifier catalogName
+
+ // A schema name pattern; must match the schema name as it is stored
+ // in the database; "" retrieves those without a schema; null means
+ // that the schema name should not be used to narrow the search.
+ 3: optional TPatternOrIdentifier schemaName
+
+ // A function name pattern; must match the function name as it is stored
+ // in the database.
+ 4: required TPatternOrIdentifier functionName
+}
+
+struct TGetFunctionsResp {
+ 1: required TStatus status
+ 2: optional TOperationHandle operationHandle
+}
+
+struct TGetPrimaryKeysReq {
+ // Session to run this request against
+ 1: required TSessionHandle sessionHandle
+
+ // Name of the catalog.
+ 2: optional TIdentifier catalogName
+
+ // Name of the schema.
+ 3: optional TIdentifier schemaName
+
+ // Name of the table.
+ 4: optional TIdentifier tableName
+}
+
+struct TGetPrimaryKeysResp {
+ 1: required TStatus status
+ 2: optional TOperationHandle operationHandle
+}
+
+struct TGetCrossReferenceReq {
+ // Session to run this request against
+ 1: required TSessionHandle sessionHandle
+
+ // Name of the parent catalog.
+ 2: optional TIdentifier parentCatalogName
+
+ // Name of the parent schema.
+ 3: optional TIdentifier parentSchemaName
+
+ // Name of the parent table.
+ 4: optional TIdentifier parentTableName
+
+ // Name of the foreign catalog.
+ 5: optional TIdentifier foreignCatalogName
+
+ // Name of the foreign schema.
+ 6: optional TIdentifier foreignSchemaName
+
+ // Name of the foreign table.
+ 7: optional TIdentifier foreignTableName
+}
+
+struct TGetCrossReferenceResp {
+ 1: required TStatus status
+ 2: optional TOperationHandle operationHandle
+}
+
+// GetOperationStatus()
+//
+// Get the status of an operation running on the server.
+struct TGetOperationStatusReq {
+ // Session to run this request against
+ 1: required TOperationHandle operationHandle
+ // optional arguments to get progress information
+ 2: optional bool getProgressUpdate
+}
+
+struct TGetOperationStatusResp {
+ 1: required TStatus status
+ 2: optional TOperationState operationState
+
+ // If operationState is ERROR_STATE, then the following fields may be set
+ // sqlState as defined in the ISO/IEF CLI specification
+ 3: optional string sqlState
+
+ // Internal error code
+ 4: optional i32 errorCode
+
+ // Error message
+ 5: optional string errorMessage
+
+ // List of statuses of sub tasks
+ 6: optional string taskStatus
+
+ // When was the operation started
+ 7: optional i64 operationStarted
+
+ // When was the operation completed
+ 8: optional i64 operationCompleted
+
+ // If the operation has the result
+ 9: optional bool hasResultSet
+
+ 10: optional TProgressUpdateResp progressUpdateResponse
+
+}
+
+
+// CancelOperation()
+//
+// Cancels processing on the specified operation handle and
+// frees any resources which were allocated.
+struct TCancelOperationReq {
+ // Operation to cancel
+ 1: required TOperationHandle operationHandle
+}
+
+struct TCancelOperationResp {
+ 1: required TStatus status
+}
+
+
+// CloseOperation()
+//
+// Given an operation in the FINISHED, CANCELED,
+// or ERROR states, CloseOperation() will free
+// all of the resources which were allocated on
+// the server to service the operation.
+struct TCloseOperationReq {
+ 1: required TOperationHandle operationHandle
+}
+
+struct TCloseOperationResp {
+ 1: required TStatus status
+}
+
+
+// GetResultSetMetadata()
+//
+// Retrieves schema information for the specified operation
+struct TGetResultSetMetadataReq {
+ // Operation for which to fetch result set schema information
+ 1: required TOperationHandle operationHandle
+}
+
+struct TGetResultSetMetadataResp {
+ 1: required TStatus status
+ 2: optional TTableSchema schema
+}
+
+
+enum TFetchOrientation {
+ // Get the next rowset. The fetch offset is ignored.
+ FETCH_NEXT,
+
+ // Get the previous rowset. The fetch offset is ignored.
+ FETCH_PRIOR,
+
+ // Return the rowset at the given fetch offset relative
+ // to the curren rowset.
+ // NOT SUPPORTED
+ FETCH_RELATIVE,
+
+ // Return the rowset at the specified fetch offset.
+ // NOT SUPPORTED
+ FETCH_ABSOLUTE,
+
+ // Get the first rowset in the result set.
+ FETCH_FIRST,
+
+ // Get the last rowset in the result set.
+ // NOT SUPPORTED
+ FETCH_LAST
+}
+
+// FetchResults()
+//
+// Fetch rows from the server corresponding to
+// a particular OperationHandle.
+struct TFetchResultsReq {
+ // Operation from which to fetch results.
+ 1: required TOperationHandle operationHandle
+
+ // The fetch orientation. For V1 this must be either
+ // FETCH_NEXT, FETCH_FIRST or FETCH_PRIOR. Defaults to FETCH_NEXT.
+ 2: required TFetchOrientation orientation = TFetchOrientation.FETCH_NEXT
+
+ // Max number of rows that should be returned in
+ // the rowset.
+ 3: required i64 maxRows
+
+ // The type of a fetch results request. 0 represents Query output. 1 represents Log
+ 4: optional i16 fetchType = 0
+}
+
+struct TFetchResultsResp {
+ 1: required TStatus status
+
+ // TRUE if there are more rows left to fetch from the server.
+ 2: optional bool hasMoreRows
+
+ // The rowset. This is optional so that we have the
+ // option in the future of adding alternate formats for
+ // representing result set data, e.g. delimited strings,
+ // binary encoded, etc.
+ 3: optional TRowSet results
+}
+
+// GetDelegationToken()
+// Retrieve delegation token for the current user
+struct TGetDelegationTokenReq {
+ // session handle
+ 1: required TSessionHandle sessionHandle
+
+ // userid for the proxy user
+ 2: required string owner
+
+ // designated renewer userid
+ 3: required string renewer
+}
+
+struct TGetDelegationTokenResp {
+ // status of the request
+ 1: required TStatus status
+
+ // delegation token string
+ 2: optional string delegationToken
+}
+
+// CancelDelegationToken()
+// Cancel the given delegation token
+struct TCancelDelegationTokenReq {
+ // session handle
+ 1: required TSessionHandle sessionHandle
+
+ // delegation token to cancel
+ 2: required string delegationToken
+}
+
+struct TCancelDelegationTokenResp {
+ // status of the request
+ 1: required TStatus status
+}
+
+// RenewDelegationToken()
+// Renew the given delegation token
+struct TRenewDelegationTokenReq {
+ // session handle
+ 1: required TSessionHandle sessionHandle
+
+ // delegation token to renew
+ 2: required string delegationToken
+}
+
+struct TRenewDelegationTokenResp {
+ // status of the request
+ 1: required TStatus status
+}
+
+enum TJobExecutionStatus {
+ IN_PROGRESS,
+ COMPLETE,
+ NOT_AVAILABLE
+}
+
+struct TProgressUpdateResp {
+ 1: required list headerNames
+ 2: required list> rows
+ 3: required double progressedPercentage
+ 4: required TJobExecutionStatus status
+ 5: required string footerSummary
+ 6: required i64 startTime
+}
+
+struct TGetQueryIdReq {
+ 1: required TOperationHandle operationHandle
+}
+
+struct TGetQueryIdResp {
+ 1: required string queryId
+}
+
+service TCLIService {
+
+ TOpenSessionResp OpenSession(1:TOpenSessionReq req);
+
+ TCloseSessionResp CloseSession(1:TCloseSessionReq req);
+
+ TGetInfoResp GetInfo(1:TGetInfoReq req);
+
+ TExecuteStatementResp ExecuteStatement(1:TExecuteStatementReq req);
+
+ TGetTypeInfoResp GetTypeInfo(1:TGetTypeInfoReq req);
+
+ TGetCatalogsResp GetCatalogs(1:TGetCatalogsReq req);
+
+ TGetSchemasResp GetSchemas(1:TGetSchemasReq req);
+
+ TGetTablesResp GetTables(1:TGetTablesReq req);
+
+ TGetTableTypesResp GetTableTypes(1:TGetTableTypesReq req);
+
+ TGetColumnsResp GetColumns(1:TGetColumnsReq req);
+
+ TGetFunctionsResp GetFunctions(1:TGetFunctionsReq req);
+
+ TGetPrimaryKeysResp GetPrimaryKeys(1:TGetPrimaryKeysReq req);
+
+ TGetCrossReferenceResp GetCrossReference(1:TGetCrossReferenceReq req);
+
+ TGetOperationStatusResp GetOperationStatus(1:TGetOperationStatusReq req);
+
+ TCancelOperationResp CancelOperation(1:TCancelOperationReq req);
+
+ TCloseOperationResp CloseOperation(1:TCloseOperationReq req);
+
+ TGetResultSetMetadataResp GetResultSetMetadata(1:TGetResultSetMetadataReq req);
+
+ TFetchResultsResp FetchResults(1:TFetchResultsReq req);
+
+ TGetDelegationTokenResp GetDelegationToken(1:TGetDelegationTokenReq req);
+
+ TCancelDelegationTokenResp CancelDelegationToken(1:TCancelDelegationTokenReq req);
+
+ TRenewDelegationTokenResp RenewDelegationToken(1:TRenewDelegationTokenReq req);
+
+ TGetQueryIdResp GetQueryId(1:TGetQueryIdReq req);
+
+ TSetClientInfoResp SetClientInfo(1:TSetClientInfoReq req);
+}
diff --git a/sql/thriftserver/pom.xml b/sql/thriftserver/pom.xml
new file mode 100644
index 0000000000000..0955c1c0009aa
--- /dev/null
+++ b/sql/thriftserver/pom.xml
@@ -0,0 +1,244 @@
+
+
+
+
+ 4.0.0
+
+ org.apache.spark
+ spark-parent_2.12
+ 3.0.0-SNAPSHOT
+ ../../pom.xml
+
+
+ spark-thriftserver_2.12
+ jar
+ Spark Project Thrift Server
+ http://spark.apache.org/
+
+ thriftserver
+
+
+
+
+ org.apache.spark
+ spark-hive_${scala.binary.version}
+ ${project.version}
+
+
+ org.apache.spark
+ spark-core_${scala.binary.version}
+ ${project.version}
+ test-jar
+ test
+
+
+ org.apache.spark
+ spark-catalyst_${scala.binary.version}
+ ${project.version}
+ test-jar
+ test
+
+
+ org.apache.spark
+ spark-hive_${scala.binary.version}
+ ${project.version}
+ test-jar
+ test
+
+
+ com.google.guava
+ guava
+
+
+ ${hive.group}
+ hive-cli
+
+
+ ${hive.group}
+ hive-jdbc
+
+
+ ${hive.group}
+ hive-beeline
+
+
+ ${hive.group}
+ hive-service
+ ${hive.version}
+ test
+
+
+ ${hive.group}
+ hive-exec
+
+
+ ${hive.group}
+ hive-metastore
+
+
+ org.eclipse.jetty.aggregate
+ jetty-all
+
+
+ commons-codec
+ commons-codec
+
+
+ commons-cli
+ commons-cli
+
+
+ commons-lang
+ commons-lang
+
+
+ commons-logging
+ commons-logging
+
+
+ org.apache.thrift
+ libfb303
+
+
+ org.apache.thrift
+ libthrift
+
+
+ org.apache.curator
+ curator-framework
+
+
+ org.apache.curator
+ curator-recipes
+
+
+ jdk.tools
+ jdk.tools
+
+
+
+
+ org.eclipse.jetty
+ jetty-server
+ provided
+
+
+ org.eclipse.jetty
+ jetty-servlet
+ provided
+
+
+
+ org.seleniumhq.selenium
+ selenium-java
+ test
+
+
+ org.seleniumhq.selenium
+ selenium-htmlunit-driver
+ test
+
+
+ org.apache.spark
+ spark-sql_${scala.binary.version}
+ test-jar
+ ${project.version}
+ test
+
+
+ org.apache.spark
+ spark-tags_${scala.binary.version}
+
+
+
+
+ org.apache.spark
+ spark-tags_${scala.binary.version}
+ test-jar
+ test
+
+
+
+ net.sf.jpam
+ jpam
+
+
+
+ target/scala-${scala.binary.version}/classes
+ target/scala-${scala.binary.version}/test-classes
+
+
+ org.codehaus.mojo
+ build-helper-maven-plugin
+
+
+ add-sources
+ generate-sources
+
+ add-source
+
+
+
+
+ ${basedir}/src/gen/thrift/gen-javabean
+
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-checkstyle-plugin
+ 3.1.0
+
+ false
+ true
+
+ ${basedir}/src/main/scala
+
+
+ ${basedir}/src/test/java
+
+ dev/checkstyle.xml
+ ${basedir}/target/checkstyle-output.xml
+ ${project.build.sourceEncoding}
+ ${project.reporting.outputEncoding}
+
+
+
+ com.puppycrawl.tools
+ checkstyle
+ 8.25
+
+
+
+
+
+ check
+
+
+
+
+
+
+
diff --git a/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TArrayTypeEntry.java b/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TArrayTypeEntry.java
new file mode 100644
index 0000000000000..792d9186d52e6
--- /dev/null
+++ b/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TArrayTypeEntry.java
@@ -0,0 +1,387 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *
+ */
+package org.apache.spark.sql.thriftserver.cli.thrift;
+
+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 TArrayTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TArrayTypeEntry");
+
+ private static final org.apache.thrift.protocol.TField OBJECT_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("objectTypePtr", org.apache.thrift.protocol.TType.I32, (short)1);
+
+ private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>();
+ static {
+ schemes.put(StandardScheme.class, new TArrayTypeEntryStandardSchemeFactory());
+ schemes.put(TupleScheme.class, new TArrayTypeEntryTupleSchemeFactory());
+ }
+
+ private int objectTypePtr; // 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 {
+ OBJECT_TYPE_PTR((short)1, "objectTypePtr");
+
+ private static final Map byName = new HashMap();
+
+ 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: // OBJECT_TYPE_PTR
+ return OBJECT_TYPE_PTR;
+ 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 __OBJECTTYPEPTR_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.OBJECT_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("objectTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr")));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TArrayTypeEntry.class, metaDataMap);
+ }
+
+ public TArrayTypeEntry() {
+ }
+
+ public TArrayTypeEntry(
+ int objectTypePtr)
+ {
+ this();
+ this.objectTypePtr = objectTypePtr;
+ setObjectTypePtrIsSet(true);
+ }
+
+ /**
+ * Performs a deep copy on other .
+ */
+ public TArrayTypeEntry(TArrayTypeEntry other) {
+ __isset_bitfield = other.__isset_bitfield;
+ this.objectTypePtr = other.objectTypePtr;
+ }
+
+ public TArrayTypeEntry deepCopy() {
+ return new TArrayTypeEntry(this);
+ }
+
+ @Override
+ public void clear() {
+ setObjectTypePtrIsSet(false);
+ this.objectTypePtr = 0;
+ }
+
+ public int getObjectTypePtr() {
+ return this.objectTypePtr;
+ }
+
+ public void setObjectTypePtr(int objectTypePtr) {
+ this.objectTypePtr = objectTypePtr;
+ setObjectTypePtrIsSet(true);
+ }
+
+ public void unsetObjectTypePtr() {
+ __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __OBJECTTYPEPTR_ISSET_ID);
+ }
+
+ /** Returns true if field objectTypePtr is set (has been assigned a value) and false otherwise */
+ public boolean isSetObjectTypePtr() {
+ return EncodingUtils.testBit(__isset_bitfield, __OBJECTTYPEPTR_ISSET_ID);
+ }
+
+ public void setObjectTypePtrIsSet(boolean value) {
+ __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __OBJECTTYPEPTR_ISSET_ID, value);
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case OBJECT_TYPE_PTR:
+ if (value == null) {
+ unsetObjectTypePtr();
+ } else {
+ setObjectTypePtr((Integer)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case OBJECT_TYPE_PTR:
+ return getObjectTypePtr();
+
+ }
+ 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 OBJECT_TYPE_PTR:
+ return isSetObjectTypePtr();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof TArrayTypeEntry)
+ return this.equals((TArrayTypeEntry)that);
+ return false;
+ }
+
+ public boolean equals(TArrayTypeEntry that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_objectTypePtr = true;
+ boolean that_present_objectTypePtr = true;
+ if (this_present_objectTypePtr || that_present_objectTypePtr) {
+ if (!(this_present_objectTypePtr && that_present_objectTypePtr))
+ return false;
+ if (this.objectTypePtr != that.objectTypePtr)
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ List list = new ArrayList();
+
+ boolean present_objectTypePtr = true;
+ list.add(present_objectTypePtr);
+ if (present_objectTypePtr)
+ list.add(objectTypePtr);
+
+ return list.hashCode();
+ }
+
+ @Override
+ public int compareTo(TArrayTypeEntry other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = Boolean.valueOf(isSetObjectTypePtr()).compareTo(other.isSetObjectTypePtr());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetObjectTypePtr()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.objectTypePtr, other.objectTypePtr);
+ 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("TArrayTypeEntry(");
+ boolean first = true;
+
+ sb.append("objectTypePtr:");
+ sb.append(this.objectTypePtr);
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ if (!isSetObjectTypePtr()) {
+ throw new org.apache.thrift.protocol.TProtocolException("Required field 'objectTypePtr' is unset! Struct:" + toString());
+ }
+
+ // check for sub-struct validity
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+ try {
+ // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+ __isset_bitfield = 0;
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class TArrayTypeEntryStandardSchemeFactory implements SchemeFactory {
+ public TArrayTypeEntryStandardScheme getScheme() {
+ return new TArrayTypeEntryStandardScheme();
+ }
+ }
+
+ private static class TArrayTypeEntryStandardScheme extends StandardScheme {
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot, TArrayTypeEntry 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: // OBJECT_TYPE_PTR
+ if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+ struct.objectTypePtr = iprot.readI32();
+ struct.setObjectTypePtrIsSet(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, TArrayTypeEntry struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ oprot.writeFieldBegin(OBJECT_TYPE_PTR_FIELD_DESC);
+ oprot.writeI32(struct.objectTypePtr);
+ oprot.writeFieldEnd();
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class TArrayTypeEntryTupleSchemeFactory implements SchemeFactory {
+ public TArrayTypeEntryTupleScheme getScheme() {
+ return new TArrayTypeEntryTupleScheme();
+ }
+ }
+
+ private static class TArrayTypeEntryTupleScheme extends TupleScheme {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, TArrayTypeEntry struct) throws org.apache.thrift.TException {
+ TTupleProtocol oprot = (TTupleProtocol) prot;
+ oprot.writeI32(struct.objectTypePtr);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, TArrayTypeEntry struct) throws org.apache.thrift.TException {
+ TTupleProtocol iprot = (TTupleProtocol) prot;
+ struct.objectTypePtr = iprot.readI32();
+ struct.setObjectTypePtrIsSet(true);
+ }
+ }
+
+}
+
diff --git a/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TBinaryColumn.java b/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TBinaryColumn.java
new file mode 100644
index 0000000000000..c75b851286b0e
--- /dev/null
+++ b/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TBinaryColumn.java
@@ -0,0 +1,548 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *
+ */
+package org.apache.spark.sql.thriftserver.cli.thrift;
+
+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 TBinaryColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBinaryColumn");
+
+ private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1);
+ private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+ private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>();
+ static {
+ schemes.put(StandardScheme.class, new TBinaryColumnStandardSchemeFactory());
+ schemes.put(TupleScheme.class, new TBinaryColumnTupleSchemeFactory());
+ }
+
+ private List values; // required
+ private ByteBuffer nulls; // 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 {
+ VALUES((short)1, "values"),
+ NULLS((short)2, "nulls");
+
+ private static final Map byName = new HashMap();
+
+ 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: // VALUES
+ return VALUES;
+ case 2: // NULLS
+ return NULLS;
+ 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.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED,
+ new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))));
+ tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBinaryColumn.class, metaDataMap);
+ }
+
+ public TBinaryColumn() {
+ }
+
+ public TBinaryColumn(
+ List values,
+ ByteBuffer nulls)
+ {
+ this();
+ this.values = values;
+ this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls);
+ }
+
+ /**
+ * Performs a deep copy on other .
+ */
+ public TBinaryColumn(TBinaryColumn other) {
+ if (other.isSetValues()) {
+ List __this__values = new ArrayList(other.values);
+ this.values = __this__values;
+ }
+ if (other.isSetNulls()) {
+ this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls);
+ }
+ }
+
+ public TBinaryColumn deepCopy() {
+ return new TBinaryColumn(this);
+ }
+
+ @Override
+ public void clear() {
+ this.values = null;
+ this.nulls = null;
+ }
+
+ public int getValuesSize() {
+ return (this.values == null) ? 0 : this.values.size();
+ }
+
+ public java.util.Iterator getValuesIterator() {
+ return (this.values == null) ? null : this.values.iterator();
+ }
+
+ public void addToValues(ByteBuffer elem) {
+ if (this.values == null) {
+ this.values = new ArrayList();
+ }
+ this.values.add(elem);
+ }
+
+ public List getValues() {
+ return this.values;
+ }
+
+ public void setValues(List values) {
+ this.values = values;
+ }
+
+ public void unsetValues() {
+ this.values = null;
+ }
+
+ /** Returns true if field values is set (has been assigned a value) and false otherwise */
+ public boolean isSetValues() {
+ return this.values != null;
+ }
+
+ public void setValuesIsSet(boolean value) {
+ if (!value) {
+ this.values = null;
+ }
+ }
+
+ public byte[] getNulls() {
+ setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls));
+ return nulls == null ? null : nulls.array();
+ }
+
+ public ByteBuffer bufferForNulls() {
+ return org.apache.thrift.TBaseHelper.copyBinary(nulls);
+ }
+
+ public void setNulls(byte[] nulls) {
+ this.nulls = nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(nulls, nulls.length));
+ }
+
+ public void setNulls(ByteBuffer nulls) {
+ this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls);
+ }
+
+ public void unsetNulls() {
+ this.nulls = null;
+ }
+
+ /** Returns true if field nulls is set (has been assigned a value) and false otherwise */
+ public boolean isSetNulls() {
+ return this.nulls != null;
+ }
+
+ public void setNullsIsSet(boolean value) {
+ if (!value) {
+ this.nulls = null;
+ }
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case VALUES:
+ if (value == null) {
+ unsetValues();
+ } else {
+ setValues((List)value);
+ }
+ break;
+
+ case NULLS:
+ if (value == null) {
+ unsetNulls();
+ } else {
+ setNulls((ByteBuffer)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case VALUES:
+ return getValues();
+
+ case NULLS:
+ return getNulls();
+
+ }
+ 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 VALUES:
+ return isSetValues();
+ case NULLS:
+ return isSetNulls();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof TBinaryColumn)
+ return this.equals((TBinaryColumn)that);
+ return false;
+ }
+
+ public boolean equals(TBinaryColumn that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_values = true && this.isSetValues();
+ boolean that_present_values = true && that.isSetValues();
+ if (this_present_values || that_present_values) {
+ if (!(this_present_values && that_present_values))
+ return false;
+ if (!this.values.equals(that.values))
+ return false;
+ }
+
+ boolean this_present_nulls = true && this.isSetNulls();
+ boolean that_present_nulls = true && that.isSetNulls();
+ if (this_present_nulls || that_present_nulls) {
+ if (!(this_present_nulls && that_present_nulls))
+ return false;
+ if (!this.nulls.equals(that.nulls))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ List list = new ArrayList();
+
+ boolean present_values = true && (isSetValues());
+ list.add(present_values);
+ if (present_values)
+ list.add(values);
+
+ boolean present_nulls = true && (isSetNulls());
+ list.add(present_nulls);
+ if (present_nulls)
+ list.add(nulls);
+
+ return list.hashCode();
+ }
+
+ @Override
+ public int compareTo(TBinaryColumn other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = Boolean.valueOf(isSetValues()).compareTo(other.isSetValues());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetValues()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, other.values);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetNulls()).compareTo(other.isSetNulls());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetNulls()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, other.nulls);
+ 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("TBinaryColumn(");
+ boolean first = true;
+
+ sb.append("values:");
+ if (this.values == null) {
+ sb.append("null");
+ } else {
+ org.apache.thrift.TBaseHelper.toString(this.values, sb);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("nulls:");
+ if (this.nulls == null) {
+ sb.append("null");
+ } else {
+ org.apache.thrift.TBaseHelper.toString(this.nulls, sb);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ if (!isSetValues()) {
+ throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString());
+ }
+
+ if (!isSetNulls()) {
+ throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString());
+ }
+
+ // check for sub-struct validity
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+ try {
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class TBinaryColumnStandardSchemeFactory implements SchemeFactory {
+ public TBinaryColumnStandardScheme getScheme() {
+ return new TBinaryColumnStandardScheme();
+ }
+ }
+
+ private static class TBinaryColumnStandardScheme extends StandardScheme {
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot, TBinaryColumn 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: // VALUES
+ if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+ {
+ org.apache.thrift.protocol.TList _list110 = iprot.readListBegin();
+ struct.values = new ArrayList(_list110.size);
+ ByteBuffer _elem111;
+ for (int _i112 = 0; _i112 < _list110.size; ++_i112)
+ {
+ _elem111 = iprot.readBinary();
+ struct.values.add(_elem111);
+ }
+ iprot.readListEnd();
+ }
+ struct.setValuesIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // NULLS
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.nulls = iprot.readBinary();
+ struct.setNullsIsSet(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, TBinaryColumn struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.values != null) {
+ oprot.writeFieldBegin(VALUES_FIELD_DESC);
+ {
+ oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size()));
+ for (ByteBuffer _iter113 : struct.values)
+ {
+ oprot.writeBinary(_iter113);
+ }
+ oprot.writeListEnd();
+ }
+ oprot.writeFieldEnd();
+ }
+ if (struct.nulls != null) {
+ oprot.writeFieldBegin(NULLS_FIELD_DESC);
+ oprot.writeBinary(struct.nulls);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class TBinaryColumnTupleSchemeFactory implements SchemeFactory {
+ public TBinaryColumnTupleScheme getScheme() {
+ return new TBinaryColumnTupleScheme();
+ }
+ }
+
+ private static class TBinaryColumnTupleScheme extends TupleScheme {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, TBinaryColumn struct) throws org.apache.thrift.TException {
+ TTupleProtocol oprot = (TTupleProtocol) prot;
+ {
+ oprot.writeI32(struct.values.size());
+ for (ByteBuffer _iter114 : struct.values)
+ {
+ oprot.writeBinary(_iter114);
+ }
+ }
+ oprot.writeBinary(struct.nulls);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, TBinaryColumn struct) throws org.apache.thrift.TException {
+ TTupleProtocol iprot = (TTupleProtocol) prot;
+ {
+ org.apache.thrift.protocol.TList _list115 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+ struct.values = new ArrayList(_list115.size);
+ ByteBuffer _elem116;
+ for (int _i117 = 0; _i117 < _list115.size; ++_i117)
+ {
+ _elem116 = iprot.readBinary();
+ struct.values.add(_elem116);
+ }
+ }
+ struct.setValuesIsSet(true);
+ struct.nulls = iprot.readBinary();
+ struct.setNullsIsSet(true);
+ }
+ }
+
+}
+
diff --git a/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TBoolColumn.java b/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TBoolColumn.java
new file mode 100644
index 0000000000000..acf64d10ce74b
--- /dev/null
+++ b/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TBoolColumn.java
@@ -0,0 +1,548 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *
+ */
+package org.apache.spark.sql.thriftserver.cli.thrift;
+
+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 TBoolColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBoolColumn");
+
+ private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1);
+ private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+ private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>();
+ static {
+ schemes.put(StandardScheme.class, new TBoolColumnStandardSchemeFactory());
+ schemes.put(TupleScheme.class, new TBoolColumnTupleSchemeFactory());
+ }
+
+ private List values; // required
+ private ByteBuffer nulls; // 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 {
+ VALUES((short)1, "values"),
+ NULLS((short)2, "nulls");
+
+ private static final Map byName = new HashMap();
+
+ 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: // VALUES
+ return VALUES;
+ case 2: // NULLS
+ return NULLS;
+ 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.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED,
+ new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))));
+ tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBoolColumn.class, metaDataMap);
+ }
+
+ public TBoolColumn() {
+ }
+
+ public TBoolColumn(
+ List values,
+ ByteBuffer nulls)
+ {
+ this();
+ this.values = values;
+ this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls);
+ }
+
+ /**
+ * Performs a deep copy on other .
+ */
+ public TBoolColumn(TBoolColumn other) {
+ if (other.isSetValues()) {
+ List __this__values = new ArrayList(other.values);
+ this.values = __this__values;
+ }
+ if (other.isSetNulls()) {
+ this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls);
+ }
+ }
+
+ public TBoolColumn deepCopy() {
+ return new TBoolColumn(this);
+ }
+
+ @Override
+ public void clear() {
+ this.values = null;
+ this.nulls = null;
+ }
+
+ public int getValuesSize() {
+ return (this.values == null) ? 0 : this.values.size();
+ }
+
+ public java.util.Iterator getValuesIterator() {
+ return (this.values == null) ? null : this.values.iterator();
+ }
+
+ public void addToValues(boolean elem) {
+ if (this.values == null) {
+ this.values = new ArrayList();
+ }
+ this.values.add(elem);
+ }
+
+ public List getValues() {
+ return this.values;
+ }
+
+ public void setValues(List values) {
+ this.values = values;
+ }
+
+ public void unsetValues() {
+ this.values = null;
+ }
+
+ /** Returns true if field values is set (has been assigned a value) and false otherwise */
+ public boolean isSetValues() {
+ return this.values != null;
+ }
+
+ public void setValuesIsSet(boolean value) {
+ if (!value) {
+ this.values = null;
+ }
+ }
+
+ public byte[] getNulls() {
+ setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls));
+ return nulls == null ? null : nulls.array();
+ }
+
+ public ByteBuffer bufferForNulls() {
+ return org.apache.thrift.TBaseHelper.copyBinary(nulls);
+ }
+
+ public void setNulls(byte[] nulls) {
+ this.nulls = nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(nulls, nulls.length));
+ }
+
+ public void setNulls(ByteBuffer nulls) {
+ this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls);
+ }
+
+ public void unsetNulls() {
+ this.nulls = null;
+ }
+
+ /** Returns true if field nulls is set (has been assigned a value) and false otherwise */
+ public boolean isSetNulls() {
+ return this.nulls != null;
+ }
+
+ public void setNullsIsSet(boolean value) {
+ if (!value) {
+ this.nulls = null;
+ }
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case VALUES:
+ if (value == null) {
+ unsetValues();
+ } else {
+ setValues((List)value);
+ }
+ break;
+
+ case NULLS:
+ if (value == null) {
+ unsetNulls();
+ } else {
+ setNulls((ByteBuffer)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case VALUES:
+ return getValues();
+
+ case NULLS:
+ return getNulls();
+
+ }
+ 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 VALUES:
+ return isSetValues();
+ case NULLS:
+ return isSetNulls();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof TBoolColumn)
+ return this.equals((TBoolColumn)that);
+ return false;
+ }
+
+ public boolean equals(TBoolColumn that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_values = true && this.isSetValues();
+ boolean that_present_values = true && that.isSetValues();
+ if (this_present_values || that_present_values) {
+ if (!(this_present_values && that_present_values))
+ return false;
+ if (!this.values.equals(that.values))
+ return false;
+ }
+
+ boolean this_present_nulls = true && this.isSetNulls();
+ boolean that_present_nulls = true && that.isSetNulls();
+ if (this_present_nulls || that_present_nulls) {
+ if (!(this_present_nulls && that_present_nulls))
+ return false;
+ if (!this.nulls.equals(that.nulls))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ List list = new ArrayList();
+
+ boolean present_values = true && (isSetValues());
+ list.add(present_values);
+ if (present_values)
+ list.add(values);
+
+ boolean present_nulls = true && (isSetNulls());
+ list.add(present_nulls);
+ if (present_nulls)
+ list.add(nulls);
+
+ return list.hashCode();
+ }
+
+ @Override
+ public int compareTo(TBoolColumn other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = Boolean.valueOf(isSetValues()).compareTo(other.isSetValues());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetValues()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, other.values);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetNulls()).compareTo(other.isSetNulls());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetNulls()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, other.nulls);
+ 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("TBoolColumn(");
+ boolean first = true;
+
+ sb.append("values:");
+ if (this.values == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.values);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("nulls:");
+ if (this.nulls == null) {
+ sb.append("null");
+ } else {
+ org.apache.thrift.TBaseHelper.toString(this.nulls, sb);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ if (!isSetValues()) {
+ throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString());
+ }
+
+ if (!isSetNulls()) {
+ throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString());
+ }
+
+ // check for sub-struct validity
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+ try {
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class TBoolColumnStandardSchemeFactory implements SchemeFactory {
+ public TBoolColumnStandardScheme getScheme() {
+ return new TBoolColumnStandardScheme();
+ }
+ }
+
+ private static class TBoolColumnStandardScheme extends StandardScheme {
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot, TBoolColumn 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: // VALUES
+ if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+ {
+ org.apache.thrift.protocol.TList _list54 = iprot.readListBegin();
+ struct.values = new ArrayList(_list54.size);
+ boolean _elem55;
+ for (int _i56 = 0; _i56 < _list54.size; ++_i56)
+ {
+ _elem55 = iprot.readBool();
+ struct.values.add(_elem55);
+ }
+ iprot.readListEnd();
+ }
+ struct.setValuesIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // NULLS
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.nulls = iprot.readBinary();
+ struct.setNullsIsSet(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, TBoolColumn struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.values != null) {
+ oprot.writeFieldBegin(VALUES_FIELD_DESC);
+ {
+ oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BOOL, struct.values.size()));
+ for (boolean _iter57 : struct.values)
+ {
+ oprot.writeBool(_iter57);
+ }
+ oprot.writeListEnd();
+ }
+ oprot.writeFieldEnd();
+ }
+ if (struct.nulls != null) {
+ oprot.writeFieldBegin(NULLS_FIELD_DESC);
+ oprot.writeBinary(struct.nulls);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class TBoolColumnTupleSchemeFactory implements SchemeFactory {
+ public TBoolColumnTupleScheme getScheme() {
+ return new TBoolColumnTupleScheme();
+ }
+ }
+
+ private static class TBoolColumnTupleScheme extends TupleScheme {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, TBoolColumn struct) throws org.apache.thrift.TException {
+ TTupleProtocol oprot = (TTupleProtocol) prot;
+ {
+ oprot.writeI32(struct.values.size());
+ for (boolean _iter58 : struct.values)
+ {
+ oprot.writeBool(_iter58);
+ }
+ }
+ oprot.writeBinary(struct.nulls);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, TBoolColumn struct) throws org.apache.thrift.TException {
+ TTupleProtocol iprot = (TTupleProtocol) prot;
+ {
+ org.apache.thrift.protocol.TList _list59 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BOOL, iprot.readI32());
+ struct.values = new ArrayList(_list59.size);
+ boolean _elem60;
+ for (int _i61 = 0; _i61 < _list59.size; ++_i61)
+ {
+ _elem60 = iprot.readBool();
+ struct.values.add(_elem60);
+ }
+ }
+ struct.setValuesIsSet(true);
+ struct.nulls = iprot.readBinary();
+ struct.setNullsIsSet(true);
+ }
+ }
+
+}
+
diff --git a/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TBoolValue.java b/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TBoolValue.java
new file mode 100644
index 0000000000000..10a9e0d267f1c
--- /dev/null
+++ b/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TBoolValue.java
@@ -0,0 +1,390 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *
+ */
+package org.apache.spark.sql.thriftserver.cli.thrift;
+
+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 TBoolValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBoolValue");
+
+ private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.BOOL, (short)1);
+
+ private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>();
+ static {
+ schemes.put(StandardScheme.class, new TBoolValueStandardSchemeFactory());
+ schemes.put(TupleScheme.class, new TBoolValueTupleSchemeFactory());
+ }
+
+ private boolean value; // optional
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ VALUE((short)1, "value");
+
+ private static final Map byName = new HashMap();
+
+ 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: // VALUE
+ return VALUE;
+ 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 __VALUE_ISSET_ID = 0;
+ private byte __isset_bitfield = 0;
+ private static final _Fields optionals[] = {_Fields.VALUE};
+ 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.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBoolValue.class, metaDataMap);
+ }
+
+ public TBoolValue() {
+ }
+
+ /**
+ * Performs a deep copy on other .
+ */
+ public TBoolValue(TBoolValue other) {
+ __isset_bitfield = other.__isset_bitfield;
+ this.value = other.value;
+ }
+
+ public TBoolValue deepCopy() {
+ return new TBoolValue(this);
+ }
+
+ @Override
+ public void clear() {
+ setValueIsSet(false);
+ this.value = false;
+ }
+
+ public boolean isValue() {
+ return this.value;
+ }
+
+ public void setValue(boolean value) {
+ this.value = value;
+ setValueIsSet(true);
+ }
+
+ public void unsetValue() {
+ __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID);
+ }
+
+ /** Returns true if field value is set (has been assigned a value) and false otherwise */
+ public boolean isSetValue() {
+ return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID);
+ }
+
+ public void setValueIsSet(boolean value) {
+ __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value);
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case VALUE:
+ if (value == null) {
+ unsetValue();
+ } else {
+ setValue((Boolean)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case VALUE:
+ return isValue();
+
+ }
+ 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 VALUE:
+ return isSetValue();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof TBoolValue)
+ return this.equals((TBoolValue)that);
+ return false;
+ }
+
+ public boolean equals(TBoolValue that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_value = true && this.isSetValue();
+ boolean that_present_value = true && that.isSetValue();
+ if (this_present_value || that_present_value) {
+ if (!(this_present_value && that_present_value))
+ return false;
+ if (this.value != that.value)
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ List list = new ArrayList();
+
+ boolean present_value = true && (isSetValue());
+ list.add(present_value);
+ if (present_value)
+ list.add(value);
+
+ return list.hashCode();
+ }
+
+ @Override
+ public int compareTo(TBoolValue other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetValue()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value);
+ 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("TBoolValue(");
+ boolean first = true;
+
+ if (isSetValue()) {
+ sb.append("value:");
+ sb.append(this.value);
+ 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 TBoolValueStandardSchemeFactory implements SchemeFactory {
+ public TBoolValueStandardScheme getScheme() {
+ return new TBoolValueStandardScheme();
+ }
+ }
+
+ private static class TBoolValueStandardScheme extends StandardScheme {
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot, TBoolValue 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: // VALUE
+ if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+ struct.value = iprot.readBool();
+ struct.setValueIsSet(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, TBoolValue struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.isSetValue()) {
+ oprot.writeFieldBegin(VALUE_FIELD_DESC);
+ oprot.writeBool(struct.value);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class TBoolValueTupleSchemeFactory implements SchemeFactory {
+ public TBoolValueTupleScheme getScheme() {
+ return new TBoolValueTupleScheme();
+ }
+ }
+
+ private static class TBoolValueTupleScheme extends TupleScheme {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, TBoolValue struct) throws org.apache.thrift.TException {
+ TTupleProtocol oprot = (TTupleProtocol) prot;
+ BitSet optionals = new BitSet();
+ if (struct.isSetValue()) {
+ optionals.set(0);
+ }
+ oprot.writeBitSet(optionals, 1);
+ if (struct.isSetValue()) {
+ oprot.writeBool(struct.value);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, TBoolValue struct) throws org.apache.thrift.TException {
+ TTupleProtocol iprot = (TTupleProtocol) prot;
+ BitSet incoming = iprot.readBitSet(1);
+ if (incoming.get(0)) {
+ struct.value = iprot.readBool();
+ struct.setValueIsSet(true);
+ }
+ }
+ }
+
+}
+
diff --git a/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TByteColumn.java b/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TByteColumn.java
new file mode 100644
index 0000000000000..c63771cc0c332
--- /dev/null
+++ b/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TByteColumn.java
@@ -0,0 +1,548 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *
+ */
+package org.apache.spark.sql.thriftserver.cli.thrift;
+
+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 TByteColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TByteColumn");
+
+ private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1);
+ private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2);
+
+ private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>();
+ static {
+ schemes.put(StandardScheme.class, new TByteColumnStandardSchemeFactory());
+ schemes.put(TupleScheme.class, new TByteColumnTupleSchemeFactory());
+ }
+
+ private List values; // required
+ private ByteBuffer nulls; // 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 {
+ VALUES((short)1, "values"),
+ NULLS((short)2, "nulls");
+
+ private static final Map byName = new HashMap();
+
+ 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: // VALUES
+ return VALUES;
+ case 2: // NULLS
+ return NULLS;
+ 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.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED,
+ new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE))));
+ tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TByteColumn.class, metaDataMap);
+ }
+
+ public TByteColumn() {
+ }
+
+ public TByteColumn(
+ List values,
+ ByteBuffer nulls)
+ {
+ this();
+ this.values = values;
+ this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls);
+ }
+
+ /**
+ * Performs a deep copy on other .
+ */
+ public TByteColumn(TByteColumn other) {
+ if (other.isSetValues()) {
+ List __this__values = new ArrayList(other.values);
+ this.values = __this__values;
+ }
+ if (other.isSetNulls()) {
+ this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls);
+ }
+ }
+
+ public TByteColumn deepCopy() {
+ return new TByteColumn(this);
+ }
+
+ @Override
+ public void clear() {
+ this.values = null;
+ this.nulls = null;
+ }
+
+ public int getValuesSize() {
+ return (this.values == null) ? 0 : this.values.size();
+ }
+
+ public java.util.Iterator getValuesIterator() {
+ return (this.values == null) ? null : this.values.iterator();
+ }
+
+ public void addToValues(byte elem) {
+ if (this.values == null) {
+ this.values = new ArrayList();
+ }
+ this.values.add(elem);
+ }
+
+ public List getValues() {
+ return this.values;
+ }
+
+ public void setValues(List values) {
+ this.values = values;
+ }
+
+ public void unsetValues() {
+ this.values = null;
+ }
+
+ /** Returns true if field values is set (has been assigned a value) and false otherwise */
+ public boolean isSetValues() {
+ return this.values != null;
+ }
+
+ public void setValuesIsSet(boolean value) {
+ if (!value) {
+ this.values = null;
+ }
+ }
+
+ public byte[] getNulls() {
+ setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls));
+ return nulls == null ? null : nulls.array();
+ }
+
+ public ByteBuffer bufferForNulls() {
+ return org.apache.thrift.TBaseHelper.copyBinary(nulls);
+ }
+
+ public void setNulls(byte[] nulls) {
+ this.nulls = nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(nulls, nulls.length));
+ }
+
+ public void setNulls(ByteBuffer nulls) {
+ this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls);
+ }
+
+ public void unsetNulls() {
+ this.nulls = null;
+ }
+
+ /** Returns true if field nulls is set (has been assigned a value) and false otherwise */
+ public boolean isSetNulls() {
+ return this.nulls != null;
+ }
+
+ public void setNullsIsSet(boolean value) {
+ if (!value) {
+ this.nulls = null;
+ }
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case VALUES:
+ if (value == null) {
+ unsetValues();
+ } else {
+ setValues((List)value);
+ }
+ break;
+
+ case NULLS:
+ if (value == null) {
+ unsetNulls();
+ } else {
+ setNulls((ByteBuffer)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case VALUES:
+ return getValues();
+
+ case NULLS:
+ return getNulls();
+
+ }
+ 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 VALUES:
+ return isSetValues();
+ case NULLS:
+ return isSetNulls();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof TByteColumn)
+ return this.equals((TByteColumn)that);
+ return false;
+ }
+
+ public boolean equals(TByteColumn that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_values = true && this.isSetValues();
+ boolean that_present_values = true && that.isSetValues();
+ if (this_present_values || that_present_values) {
+ if (!(this_present_values && that_present_values))
+ return false;
+ if (!this.values.equals(that.values))
+ return false;
+ }
+
+ boolean this_present_nulls = true && this.isSetNulls();
+ boolean that_present_nulls = true && that.isSetNulls();
+ if (this_present_nulls || that_present_nulls) {
+ if (!(this_present_nulls && that_present_nulls))
+ return false;
+ if (!this.nulls.equals(that.nulls))
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ List list = new ArrayList();
+
+ boolean present_values = true && (isSetValues());
+ list.add(present_values);
+ if (present_values)
+ list.add(values);
+
+ boolean present_nulls = true && (isSetNulls());
+ list.add(present_nulls);
+ if (present_nulls)
+ list.add(nulls);
+
+ return list.hashCode();
+ }
+
+ @Override
+ public int compareTo(TByteColumn other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = Boolean.valueOf(isSetValues()).compareTo(other.isSetValues());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetValues()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, other.values);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
+ lastComparison = Boolean.valueOf(isSetNulls()).compareTo(other.isSetNulls());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetNulls()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, other.nulls);
+ 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("TByteColumn(");
+ boolean first = true;
+
+ sb.append("values:");
+ if (this.values == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.values);
+ }
+ first = false;
+ if (!first) sb.append(", ");
+ sb.append("nulls:");
+ if (this.nulls == null) {
+ sb.append("null");
+ } else {
+ org.apache.thrift.TBaseHelper.toString(this.nulls, sb);
+ }
+ first = false;
+ sb.append(")");
+ return sb.toString();
+ }
+
+ public void validate() throws org.apache.thrift.TException {
+ // check for required fields
+ if (!isSetValues()) {
+ throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString());
+ }
+
+ if (!isSetNulls()) {
+ throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString());
+ }
+
+ // check for sub-struct validity
+ }
+
+ private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+ try {
+ write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+ try {
+ read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+ } catch (org.apache.thrift.TException te) {
+ throw new java.io.IOException(te);
+ }
+ }
+
+ private static class TByteColumnStandardSchemeFactory implements SchemeFactory {
+ public TByteColumnStandardScheme getScheme() {
+ return new TByteColumnStandardScheme();
+ }
+ }
+
+ private static class TByteColumnStandardScheme extends StandardScheme {
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot, TByteColumn 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: // VALUES
+ if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+ {
+ org.apache.thrift.protocol.TList _list62 = iprot.readListBegin();
+ struct.values = new ArrayList(_list62.size);
+ byte _elem63;
+ for (int _i64 = 0; _i64 < _list62.size; ++_i64)
+ {
+ _elem63 = iprot.readByte();
+ struct.values.add(_elem63);
+ }
+ iprot.readListEnd();
+ }
+ struct.setValuesIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
+ case 2: // NULLS
+ if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+ struct.nulls = iprot.readBinary();
+ struct.setNullsIsSet(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, TByteColumn struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.values != null) {
+ oprot.writeFieldBegin(VALUES_FIELD_DESC);
+ {
+ oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BYTE, struct.values.size()));
+ for (byte _iter65 : struct.values)
+ {
+ oprot.writeByte(_iter65);
+ }
+ oprot.writeListEnd();
+ }
+ oprot.writeFieldEnd();
+ }
+ if (struct.nulls != null) {
+ oprot.writeFieldBegin(NULLS_FIELD_DESC);
+ oprot.writeBinary(struct.nulls);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class TByteColumnTupleSchemeFactory implements SchemeFactory {
+ public TByteColumnTupleScheme getScheme() {
+ return new TByteColumnTupleScheme();
+ }
+ }
+
+ private static class TByteColumnTupleScheme extends TupleScheme {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, TByteColumn struct) throws org.apache.thrift.TException {
+ TTupleProtocol oprot = (TTupleProtocol) prot;
+ {
+ oprot.writeI32(struct.values.size());
+ for (byte _iter66 : struct.values)
+ {
+ oprot.writeByte(_iter66);
+ }
+ }
+ oprot.writeBinary(struct.nulls);
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, TByteColumn struct) throws org.apache.thrift.TException {
+ TTupleProtocol iprot = (TTupleProtocol) prot;
+ {
+ org.apache.thrift.protocol.TList _list67 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BYTE, iprot.readI32());
+ struct.values = new ArrayList(_list67.size);
+ byte _elem68;
+ for (int _i69 = 0; _i69 < _list67.size; ++_i69)
+ {
+ _elem68 = iprot.readByte();
+ struct.values.add(_elem68);
+ }
+ }
+ struct.setValuesIsSet(true);
+ struct.nulls = iprot.readBinary();
+ struct.setNullsIsSet(true);
+ }
+ }
+
+}
+
diff --git a/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TByteValue.java b/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TByteValue.java
new file mode 100644
index 0000000000000..bc773e7716c2e
--- /dev/null
+++ b/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TByteValue.java
@@ -0,0 +1,390 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *
+ */
+package org.apache.spark.sql.thriftserver.cli.thrift;
+
+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 TByteValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable {
+ private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TByteValue");
+
+ private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.BYTE, (short)1);
+
+ private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>();
+ static {
+ schemes.put(StandardScheme.class, new TByteValueStandardSchemeFactory());
+ schemes.put(TupleScheme.class, new TByteValueTupleSchemeFactory());
+ }
+
+ private byte value; // optional
+
+ /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+ public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+ VALUE((short)1, "value");
+
+ private static final Map byName = new HashMap();
+
+ 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: // VALUE
+ return VALUE;
+ 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 __VALUE_ISSET_ID = 0;
+ private byte __isset_bitfield = 0;
+ private static final _Fields optionals[] = {_Fields.VALUE};
+ 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.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+ new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE)));
+ metaDataMap = Collections.unmodifiableMap(tmpMap);
+ org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TByteValue.class, metaDataMap);
+ }
+
+ public TByteValue() {
+ }
+
+ /**
+ * Performs a deep copy on other .
+ */
+ public TByteValue(TByteValue other) {
+ __isset_bitfield = other.__isset_bitfield;
+ this.value = other.value;
+ }
+
+ public TByteValue deepCopy() {
+ return new TByteValue(this);
+ }
+
+ @Override
+ public void clear() {
+ setValueIsSet(false);
+ this.value = 0;
+ }
+
+ public byte getValue() {
+ return this.value;
+ }
+
+ public void setValue(byte value) {
+ this.value = value;
+ setValueIsSet(true);
+ }
+
+ public void unsetValue() {
+ __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID);
+ }
+
+ /** Returns true if field value is set (has been assigned a value) and false otherwise */
+ public boolean isSetValue() {
+ return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID);
+ }
+
+ public void setValueIsSet(boolean value) {
+ __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value);
+ }
+
+ public void setFieldValue(_Fields field, Object value) {
+ switch (field) {
+ case VALUE:
+ if (value == null) {
+ unsetValue();
+ } else {
+ setValue((Byte)value);
+ }
+ break;
+
+ }
+ }
+
+ public Object getFieldValue(_Fields field) {
+ switch (field) {
+ case VALUE:
+ return getValue();
+
+ }
+ 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 VALUE:
+ return isSetValue();
+ }
+ throw new IllegalStateException();
+ }
+
+ @Override
+ public boolean equals(Object that) {
+ if (that == null)
+ return false;
+ if (that instanceof TByteValue)
+ return this.equals((TByteValue)that);
+ return false;
+ }
+
+ public boolean equals(TByteValue that) {
+ if (that == null)
+ return false;
+
+ boolean this_present_value = true && this.isSetValue();
+ boolean that_present_value = true && that.isSetValue();
+ if (this_present_value || that_present_value) {
+ if (!(this_present_value && that_present_value))
+ return false;
+ if (this.value != that.value)
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ List list = new ArrayList();
+
+ boolean present_value = true && (isSetValue());
+ list.add(present_value);
+ if (present_value)
+ list.add(value);
+
+ return list.hashCode();
+ }
+
+ @Override
+ public int compareTo(TByteValue other) {
+ if (!getClass().equals(other.getClass())) {
+ return getClass().getName().compareTo(other.getClass().getName());
+ }
+
+ int lastComparison = 0;
+
+ lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetValue()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value);
+ 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("TByteValue(");
+ boolean first = true;
+
+ if (isSetValue()) {
+ sb.append("value:");
+ sb.append(this.value);
+ 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 TByteValueStandardSchemeFactory implements SchemeFactory {
+ public TByteValueStandardScheme getScheme() {
+ return new TByteValueStandardScheme();
+ }
+ }
+
+ private static class TByteValueStandardScheme extends StandardScheme {
+
+ public void read(org.apache.thrift.protocol.TProtocol iprot, TByteValue 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: // VALUE
+ if (schemeField.type == org.apache.thrift.protocol.TType.BYTE) {
+ struct.value = iprot.readByte();
+ struct.setValueIsSet(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, TByteValue struct) throws org.apache.thrift.TException {
+ struct.validate();
+
+ oprot.writeStructBegin(STRUCT_DESC);
+ if (struct.isSetValue()) {
+ oprot.writeFieldBegin(VALUE_FIELD_DESC);
+ oprot.writeByte(struct.value);
+ oprot.writeFieldEnd();
+ }
+ oprot.writeFieldStop();
+ oprot.writeStructEnd();
+ }
+
+ }
+
+ private static class TByteValueTupleSchemeFactory implements SchemeFactory {
+ public TByteValueTupleScheme getScheme() {
+ return new TByteValueTupleScheme();
+ }
+ }
+
+ private static class TByteValueTupleScheme extends TupleScheme {
+
+ @Override
+ public void write(org.apache.thrift.protocol.TProtocol prot, TByteValue struct) throws org.apache.thrift.TException {
+ TTupleProtocol oprot = (TTupleProtocol) prot;
+ BitSet optionals = new BitSet();
+ if (struct.isSetValue()) {
+ optionals.set(0);
+ }
+ oprot.writeBitSet(optionals, 1);
+ if (struct.isSetValue()) {
+ oprot.writeByte(struct.value);
+ }
+ }
+
+ @Override
+ public void read(org.apache.thrift.protocol.TProtocol prot, TByteValue struct) throws org.apache.thrift.TException {
+ TTupleProtocol iprot = (TTupleProtocol) prot;
+ BitSet incoming = iprot.readBitSet(1);
+ if (incoming.get(0)) {
+ struct.value = iprot.readByte();
+ struct.setValueIsSet(true);
+ }
+ }
+ }
+
+}
+
diff --git a/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TCLIService.java b/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TCLIService.java
new file mode 100644
index 0000000000000..c27835b744cb7
--- /dev/null
+++ b/sql/thriftserver/src/gen/thrift/gen-javabean/org/apache/spark/sql/thriftserver/cli/thrift/TCLIService.java
@@ -0,0 +1,19855 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *
+ */
+package org.apache.spark.sql.thriftserver.cli.thrift;
+
+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 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 TCLIService {
+
+ public interface Iface {
+
+ public TOpenSessionResp OpenSession(TOpenSessionReq req) throws org.apache.thrift.TException;
+
+ public TCloseSessionResp CloseSession(TCloseSessionReq req) throws org.apache.thrift.TException;
+
+ public TGetInfoResp GetInfo(TGetInfoReq req) throws org.apache.thrift.TException;
+
+ public TExecuteStatementResp ExecuteStatement(TExecuteStatementReq req) throws org.apache.thrift.TException;
+
+ public TGetTypeInfoResp GetTypeInfo(TGetTypeInfoReq req) throws org.apache.thrift.TException;
+
+ public TGetCatalogsResp GetCatalogs(TGetCatalogsReq req) throws org.apache.thrift.TException;
+
+ public TGetSchemasResp GetSchemas(TGetSchemasReq req) throws org.apache.thrift.TException;
+
+ public TGetTablesResp GetTables(TGetTablesReq req) throws org.apache.thrift.TException;
+
+ public TGetTableTypesResp GetTableTypes(TGetTableTypesReq req) throws org.apache.thrift.TException;
+
+ public TGetColumnsResp GetColumns(TGetColumnsReq req) throws org.apache.thrift.TException;
+
+ public TGetFunctionsResp GetFunctions(TGetFunctionsReq req) throws org.apache.thrift.TException;
+
+ public TGetPrimaryKeysResp GetPrimaryKeys(TGetPrimaryKeysReq req) throws org.apache.thrift.TException;
+
+ public TGetCrossReferenceResp GetCrossReference(TGetCrossReferenceReq req) throws org.apache.thrift.TException;
+
+ public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) throws org.apache.thrift.TException;
+
+ public TCancelOperationResp CancelOperation(TCancelOperationReq req) throws org.apache.thrift.TException;
+
+ public TCloseOperationResp CloseOperation(TCloseOperationReq req) throws org.apache.thrift.TException;
+
+ public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq req) throws org.apache.thrift.TException;
+
+ public TFetchResultsResp FetchResults(TFetchResultsReq req) throws org.apache.thrift.TException;
+
+ public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) throws org.apache.thrift.TException;
+
+ public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) throws org.apache.thrift.TException;
+
+ public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException;
+
+ public TGetQueryIdResp GetQueryId(TGetQueryIdReq req) throws org.apache.thrift.TException;
+
+ public TSetClientInfoResp SetClientInfo(TSetClientInfoReq req) throws org.apache.thrift.TException;
+
+ }
+
+ public interface AsyncIface {
+
+ public void OpenSession(TOpenSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void CloseSession(TCloseSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void GetInfo(TGetInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void ExecuteStatement(TExecuteStatementReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void GetTypeInfo(TGetTypeInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void GetCatalogs(TGetCatalogsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void GetSchemas(TGetSchemasReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void GetTables(TGetTablesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void GetTableTypes(TGetTableTypesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void GetColumns(TGetColumnsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void GetFunctions(TGetFunctionsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void GetPrimaryKeys(TGetPrimaryKeysReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void GetCrossReference(TGetCrossReferenceReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void GetOperationStatus(TGetOperationStatusReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void CancelOperation(TCancelOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void CloseOperation(TCloseOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void GetResultSetMetadata(TGetResultSetMetadataReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void FetchResults(TFetchResultsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void GetDelegationToken(TGetDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void CancelDelegationToken(TCancelDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void RenewDelegationToken(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void GetQueryId(TGetQueryIdReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ public void SetClientInfo(TSetClientInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+ }
+
+ public static class Client extends org.apache.thrift.TServiceClient implements Iface {
+ public static class Factory implements org.apache.thrift.TServiceClientFactory {
+ 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 TOpenSessionResp OpenSession(TOpenSessionReq req) throws org.apache.thrift.TException
+ {
+ send_OpenSession(req);
+ return recv_OpenSession();
+ }
+
+ public void send_OpenSession(TOpenSessionReq req) throws org.apache.thrift.TException
+ {
+ OpenSession_args args = new OpenSession_args();
+ args.setReq(req);
+ sendBase("OpenSession", args);
+ }
+
+ public TOpenSessionResp recv_OpenSession() throws org.apache.thrift.TException
+ {
+ OpenSession_result result = new OpenSession_result();
+ receiveBase(result, "OpenSession");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "OpenSession failed: unknown result");
+ }
+
+ public TCloseSessionResp CloseSession(TCloseSessionReq req) throws org.apache.thrift.TException
+ {
+ send_CloseSession(req);
+ return recv_CloseSession();
+ }
+
+ public void send_CloseSession(TCloseSessionReq req) throws org.apache.thrift.TException
+ {
+ CloseSession_args args = new CloseSession_args();
+ args.setReq(req);
+ sendBase("CloseSession", args);
+ }
+
+ public TCloseSessionResp recv_CloseSession() throws org.apache.thrift.TException
+ {
+ CloseSession_result result = new CloseSession_result();
+ receiveBase(result, "CloseSession");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CloseSession failed: unknown result");
+ }
+
+ public TGetInfoResp GetInfo(TGetInfoReq req) throws org.apache.thrift.TException
+ {
+ send_GetInfo(req);
+ return recv_GetInfo();
+ }
+
+ public void send_GetInfo(TGetInfoReq req) throws org.apache.thrift.TException
+ {
+ GetInfo_args args = new GetInfo_args();
+ args.setReq(req);
+ sendBase("GetInfo", args);
+ }
+
+ public TGetInfoResp recv_GetInfo() throws org.apache.thrift.TException
+ {
+ GetInfo_result result = new GetInfo_result();
+ receiveBase(result, "GetInfo");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetInfo failed: unknown result");
+ }
+
+ public TExecuteStatementResp ExecuteStatement(TExecuteStatementReq req) throws org.apache.thrift.TException
+ {
+ send_ExecuteStatement(req);
+ return recv_ExecuteStatement();
+ }
+
+ public void send_ExecuteStatement(TExecuteStatementReq req) throws org.apache.thrift.TException
+ {
+ ExecuteStatement_args args = new ExecuteStatement_args();
+ args.setReq(req);
+ sendBase("ExecuteStatement", args);
+ }
+
+ public TExecuteStatementResp recv_ExecuteStatement() throws org.apache.thrift.TException
+ {
+ ExecuteStatement_result result = new ExecuteStatement_result();
+ receiveBase(result, "ExecuteStatement");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "ExecuteStatement failed: unknown result");
+ }
+
+ public TGetTypeInfoResp GetTypeInfo(TGetTypeInfoReq req) throws org.apache.thrift.TException
+ {
+ send_GetTypeInfo(req);
+ return recv_GetTypeInfo();
+ }
+
+ public void send_GetTypeInfo(TGetTypeInfoReq req) throws org.apache.thrift.TException
+ {
+ GetTypeInfo_args args = new GetTypeInfo_args();
+ args.setReq(req);
+ sendBase("GetTypeInfo", args);
+ }
+
+ public TGetTypeInfoResp recv_GetTypeInfo() throws org.apache.thrift.TException
+ {
+ GetTypeInfo_result result = new GetTypeInfo_result();
+ receiveBase(result, "GetTypeInfo");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetTypeInfo failed: unknown result");
+ }
+
+ public TGetCatalogsResp GetCatalogs(TGetCatalogsReq req) throws org.apache.thrift.TException
+ {
+ send_GetCatalogs(req);
+ return recv_GetCatalogs();
+ }
+
+ public void send_GetCatalogs(TGetCatalogsReq req) throws org.apache.thrift.TException
+ {
+ GetCatalogs_args args = new GetCatalogs_args();
+ args.setReq(req);
+ sendBase("GetCatalogs", args);
+ }
+
+ public TGetCatalogsResp recv_GetCatalogs() throws org.apache.thrift.TException
+ {
+ GetCatalogs_result result = new GetCatalogs_result();
+ receiveBase(result, "GetCatalogs");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetCatalogs failed: unknown result");
+ }
+
+ public TGetSchemasResp GetSchemas(TGetSchemasReq req) throws org.apache.thrift.TException
+ {
+ send_GetSchemas(req);
+ return recv_GetSchemas();
+ }
+
+ public void send_GetSchemas(TGetSchemasReq req) throws org.apache.thrift.TException
+ {
+ GetSchemas_args args = new GetSchemas_args();
+ args.setReq(req);
+ sendBase("GetSchemas", args);
+ }
+
+ public TGetSchemasResp recv_GetSchemas() throws org.apache.thrift.TException
+ {
+ GetSchemas_result result = new GetSchemas_result();
+ receiveBase(result, "GetSchemas");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetSchemas failed: unknown result");
+ }
+
+ public TGetTablesResp GetTables(TGetTablesReq req) throws org.apache.thrift.TException
+ {
+ send_GetTables(req);
+ return recv_GetTables();
+ }
+
+ public void send_GetTables(TGetTablesReq req) throws org.apache.thrift.TException
+ {
+ GetTables_args args = new GetTables_args();
+ args.setReq(req);
+ sendBase("GetTables", args);
+ }
+
+ public TGetTablesResp recv_GetTables() throws org.apache.thrift.TException
+ {
+ GetTables_result result = new GetTables_result();
+ receiveBase(result, "GetTables");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetTables failed: unknown result");
+ }
+
+ public TGetTableTypesResp GetTableTypes(TGetTableTypesReq req) throws org.apache.thrift.TException
+ {
+ send_GetTableTypes(req);
+ return recv_GetTableTypes();
+ }
+
+ public void send_GetTableTypes(TGetTableTypesReq req) throws org.apache.thrift.TException
+ {
+ GetTableTypes_args args = new GetTableTypes_args();
+ args.setReq(req);
+ sendBase("GetTableTypes", args);
+ }
+
+ public TGetTableTypesResp recv_GetTableTypes() throws org.apache.thrift.TException
+ {
+ GetTableTypes_result result = new GetTableTypes_result();
+ receiveBase(result, "GetTableTypes");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetTableTypes failed: unknown result");
+ }
+
+ public TGetColumnsResp GetColumns(TGetColumnsReq req) throws org.apache.thrift.TException
+ {
+ send_GetColumns(req);
+ return recv_GetColumns();
+ }
+
+ public void send_GetColumns(TGetColumnsReq req) throws org.apache.thrift.TException
+ {
+ GetColumns_args args = new GetColumns_args();
+ args.setReq(req);
+ sendBase("GetColumns", args);
+ }
+
+ public TGetColumnsResp recv_GetColumns() throws org.apache.thrift.TException
+ {
+ GetColumns_result result = new GetColumns_result();
+ receiveBase(result, "GetColumns");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetColumns failed: unknown result");
+ }
+
+ public TGetFunctionsResp GetFunctions(TGetFunctionsReq req) throws org.apache.thrift.TException
+ {
+ send_GetFunctions(req);
+ return recv_GetFunctions();
+ }
+
+ public void send_GetFunctions(TGetFunctionsReq req) throws org.apache.thrift.TException
+ {
+ GetFunctions_args args = new GetFunctions_args();
+ args.setReq(req);
+ sendBase("GetFunctions", args);
+ }
+
+ public TGetFunctionsResp recv_GetFunctions() throws org.apache.thrift.TException
+ {
+ GetFunctions_result result = new GetFunctions_result();
+ receiveBase(result, "GetFunctions");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetFunctions failed: unknown result");
+ }
+
+ public TGetPrimaryKeysResp GetPrimaryKeys(TGetPrimaryKeysReq req) throws org.apache.thrift.TException
+ {
+ send_GetPrimaryKeys(req);
+ return recv_GetPrimaryKeys();
+ }
+
+ public void send_GetPrimaryKeys(TGetPrimaryKeysReq req) throws org.apache.thrift.TException
+ {
+ GetPrimaryKeys_args args = new GetPrimaryKeys_args();
+ args.setReq(req);
+ sendBase("GetPrimaryKeys", args);
+ }
+
+ public TGetPrimaryKeysResp recv_GetPrimaryKeys() throws org.apache.thrift.TException
+ {
+ GetPrimaryKeys_result result = new GetPrimaryKeys_result();
+ receiveBase(result, "GetPrimaryKeys");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetPrimaryKeys failed: unknown result");
+ }
+
+ public TGetCrossReferenceResp GetCrossReference(TGetCrossReferenceReq req) throws org.apache.thrift.TException
+ {
+ send_GetCrossReference(req);
+ return recv_GetCrossReference();
+ }
+
+ public void send_GetCrossReference(TGetCrossReferenceReq req) throws org.apache.thrift.TException
+ {
+ GetCrossReference_args args = new GetCrossReference_args();
+ args.setReq(req);
+ sendBase("GetCrossReference", args);
+ }
+
+ public TGetCrossReferenceResp recv_GetCrossReference() throws org.apache.thrift.TException
+ {
+ GetCrossReference_result result = new GetCrossReference_result();
+ receiveBase(result, "GetCrossReference");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetCrossReference failed: unknown result");
+ }
+
+ public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) throws org.apache.thrift.TException
+ {
+ send_GetOperationStatus(req);
+ return recv_GetOperationStatus();
+ }
+
+ public void send_GetOperationStatus(TGetOperationStatusReq req) throws org.apache.thrift.TException
+ {
+ GetOperationStatus_args args = new GetOperationStatus_args();
+ args.setReq(req);
+ sendBase("GetOperationStatus", args);
+ }
+
+ public TGetOperationStatusResp recv_GetOperationStatus() throws org.apache.thrift.TException
+ {
+ GetOperationStatus_result result = new GetOperationStatus_result();
+ receiveBase(result, "GetOperationStatus");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetOperationStatus failed: unknown result");
+ }
+
+ public TCancelOperationResp CancelOperation(TCancelOperationReq req) throws org.apache.thrift.TException
+ {
+ send_CancelOperation(req);
+ return recv_CancelOperation();
+ }
+
+ public void send_CancelOperation(TCancelOperationReq req) throws org.apache.thrift.TException
+ {
+ CancelOperation_args args = new CancelOperation_args();
+ args.setReq(req);
+ sendBase("CancelOperation", args);
+ }
+
+ public TCancelOperationResp recv_CancelOperation() throws org.apache.thrift.TException
+ {
+ CancelOperation_result result = new CancelOperation_result();
+ receiveBase(result, "CancelOperation");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CancelOperation failed: unknown result");
+ }
+
+ public TCloseOperationResp CloseOperation(TCloseOperationReq req) throws org.apache.thrift.TException
+ {
+ send_CloseOperation(req);
+ return recv_CloseOperation();
+ }
+
+ public void send_CloseOperation(TCloseOperationReq req) throws org.apache.thrift.TException
+ {
+ CloseOperation_args args = new CloseOperation_args();
+ args.setReq(req);
+ sendBase("CloseOperation", args);
+ }
+
+ public TCloseOperationResp recv_CloseOperation() throws org.apache.thrift.TException
+ {
+ CloseOperation_result result = new CloseOperation_result();
+ receiveBase(result, "CloseOperation");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CloseOperation failed: unknown result");
+ }
+
+ public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq req) throws org.apache.thrift.TException
+ {
+ send_GetResultSetMetadata(req);
+ return recv_GetResultSetMetadata();
+ }
+
+ public void send_GetResultSetMetadata(TGetResultSetMetadataReq req) throws org.apache.thrift.TException
+ {
+ GetResultSetMetadata_args args = new GetResultSetMetadata_args();
+ args.setReq(req);
+ sendBase("GetResultSetMetadata", args);
+ }
+
+ public TGetResultSetMetadataResp recv_GetResultSetMetadata() throws org.apache.thrift.TException
+ {
+ GetResultSetMetadata_result result = new GetResultSetMetadata_result();
+ receiveBase(result, "GetResultSetMetadata");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetResultSetMetadata failed: unknown result");
+ }
+
+ public TFetchResultsResp FetchResults(TFetchResultsReq req) throws org.apache.thrift.TException
+ {
+ send_FetchResults(req);
+ return recv_FetchResults();
+ }
+
+ public void send_FetchResults(TFetchResultsReq req) throws org.apache.thrift.TException
+ {
+ FetchResults_args args = new FetchResults_args();
+ args.setReq(req);
+ sendBase("FetchResults", args);
+ }
+
+ public TFetchResultsResp recv_FetchResults() throws org.apache.thrift.TException
+ {
+ FetchResults_result result = new FetchResults_result();
+ receiveBase(result, "FetchResults");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "FetchResults failed: unknown result");
+ }
+
+ public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) throws org.apache.thrift.TException
+ {
+ send_GetDelegationToken(req);
+ return recv_GetDelegationToken();
+ }
+
+ public void send_GetDelegationToken(TGetDelegationTokenReq req) throws org.apache.thrift.TException
+ {
+ GetDelegationToken_args args = new GetDelegationToken_args();
+ args.setReq(req);
+ sendBase("GetDelegationToken", args);
+ }
+
+ public TGetDelegationTokenResp recv_GetDelegationToken() throws org.apache.thrift.TException
+ {
+ GetDelegationToken_result result = new GetDelegationToken_result();
+ receiveBase(result, "GetDelegationToken");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetDelegationToken failed: unknown result");
+ }
+
+ public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) throws org.apache.thrift.TException
+ {
+ send_CancelDelegationToken(req);
+ return recv_CancelDelegationToken();
+ }
+
+ public void send_CancelDelegationToken(TCancelDelegationTokenReq req) throws org.apache.thrift.TException
+ {
+ CancelDelegationToken_args args = new CancelDelegationToken_args();
+ args.setReq(req);
+ sendBase("CancelDelegationToken", args);
+ }
+
+ public TCancelDelegationTokenResp recv_CancelDelegationToken() throws org.apache.thrift.TException
+ {
+ CancelDelegationToken_result result = new CancelDelegationToken_result();
+ receiveBase(result, "CancelDelegationToken");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CancelDelegationToken failed: unknown result");
+ }
+
+ public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException
+ {
+ send_RenewDelegationToken(req);
+ return recv_RenewDelegationToken();
+ }
+
+ public void send_RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException
+ {
+ RenewDelegationToken_args args = new RenewDelegationToken_args();
+ args.setReq(req);
+ sendBase("RenewDelegationToken", args);
+ }
+
+ public TRenewDelegationTokenResp recv_RenewDelegationToken() throws org.apache.thrift.TException
+ {
+ RenewDelegationToken_result result = new RenewDelegationToken_result();
+ receiveBase(result, "RenewDelegationToken");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "RenewDelegationToken failed: unknown result");
+ }
+
+ public TGetQueryIdResp GetQueryId(TGetQueryIdReq req) throws org.apache.thrift.TException
+ {
+ send_GetQueryId(req);
+ return recv_GetQueryId();
+ }
+
+ public void send_GetQueryId(TGetQueryIdReq req) throws org.apache.thrift.TException
+ {
+ GetQueryId_args args = new GetQueryId_args();
+ args.setReq(req);
+ sendBase("GetQueryId", args);
+ }
+
+ public TGetQueryIdResp recv_GetQueryId() throws org.apache.thrift.TException
+ {
+ GetQueryId_result result = new GetQueryId_result();
+ receiveBase(result, "GetQueryId");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetQueryId failed: unknown result");
+ }
+
+ public TSetClientInfoResp SetClientInfo(TSetClientInfoReq req) throws org.apache.thrift.TException
+ {
+ send_SetClientInfo(req);
+ return recv_SetClientInfo();
+ }
+
+ public void send_SetClientInfo(TSetClientInfoReq req) throws org.apache.thrift.TException
+ {
+ SetClientInfo_args args = new SetClientInfo_args();
+ args.setReq(req);
+ sendBase("SetClientInfo", args);
+ }
+
+ public TSetClientInfoResp recv_SetClientInfo() throws org.apache.thrift.TException
+ {
+ SetClientInfo_result result = new SetClientInfo_result();
+ receiveBase(result, "SetClientInfo");
+ if (result.isSetSuccess()) {
+ return result.success;
+ }
+ throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "SetClientInfo failed: unknown result");
+ }
+
+ }
+ public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
+ public static class Factory implements org.apache.thrift.async.TAsyncClientFactory {
+ 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 OpenSession(TOpenSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ OpenSession_call method_call = new OpenSession_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class OpenSession_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TOpenSessionReq req;
+ public OpenSession_call(TOpenSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("OpenSession", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ OpenSession_args args = new OpenSession_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TOpenSessionResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_OpenSession();
+ }
+ }
+
+ public void CloseSession(TCloseSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ CloseSession_call method_call = new CloseSession_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class CloseSession_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TCloseSessionReq req;
+ public CloseSession_call(TCloseSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CloseSession", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ CloseSession_args args = new CloseSession_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TCloseSessionResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_CloseSession();
+ }
+ }
+
+ public void GetInfo(TGetInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ GetInfo_call method_call = new GetInfo_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class GetInfo_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TGetInfoReq req;
+ public GetInfo_call(TGetInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetInfo", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ GetInfo_args args = new GetInfo_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TGetInfoResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_GetInfo();
+ }
+ }
+
+ public void ExecuteStatement(TExecuteStatementReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ ExecuteStatement_call method_call = new ExecuteStatement_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class ExecuteStatement_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TExecuteStatementReq req;
+ public ExecuteStatement_call(TExecuteStatementReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("ExecuteStatement", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ ExecuteStatement_args args = new ExecuteStatement_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TExecuteStatementResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_ExecuteStatement();
+ }
+ }
+
+ public void GetTypeInfo(TGetTypeInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ GetTypeInfo_call method_call = new GetTypeInfo_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class GetTypeInfo_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TGetTypeInfoReq req;
+ public GetTypeInfo_call(TGetTypeInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetTypeInfo", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ GetTypeInfo_args args = new GetTypeInfo_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TGetTypeInfoResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_GetTypeInfo();
+ }
+ }
+
+ public void GetCatalogs(TGetCatalogsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ GetCatalogs_call method_call = new GetCatalogs_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class GetCatalogs_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TGetCatalogsReq req;
+ public GetCatalogs_call(TGetCatalogsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetCatalogs", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ GetCatalogs_args args = new GetCatalogs_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TGetCatalogsResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_GetCatalogs();
+ }
+ }
+
+ public void GetSchemas(TGetSchemasReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ GetSchemas_call method_call = new GetSchemas_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class GetSchemas_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TGetSchemasReq req;
+ public GetSchemas_call(TGetSchemasReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetSchemas", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ GetSchemas_args args = new GetSchemas_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TGetSchemasResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_GetSchemas();
+ }
+ }
+
+ public void GetTables(TGetTablesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ GetTables_call method_call = new GetTables_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class GetTables_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TGetTablesReq req;
+ public GetTables_call(TGetTablesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetTables", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ GetTables_args args = new GetTables_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TGetTablesResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_GetTables();
+ }
+ }
+
+ public void GetTableTypes(TGetTableTypesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ GetTableTypes_call method_call = new GetTableTypes_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class GetTableTypes_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TGetTableTypesReq req;
+ public GetTableTypes_call(TGetTableTypesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetTableTypes", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ GetTableTypes_args args = new GetTableTypes_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TGetTableTypesResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_GetTableTypes();
+ }
+ }
+
+ public void GetColumns(TGetColumnsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ GetColumns_call method_call = new GetColumns_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class GetColumns_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TGetColumnsReq req;
+ public GetColumns_call(TGetColumnsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetColumns", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ GetColumns_args args = new GetColumns_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TGetColumnsResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_GetColumns();
+ }
+ }
+
+ public void GetFunctions(TGetFunctionsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ GetFunctions_call method_call = new GetFunctions_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class GetFunctions_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TGetFunctionsReq req;
+ public GetFunctions_call(TGetFunctionsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetFunctions", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ GetFunctions_args args = new GetFunctions_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TGetFunctionsResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_GetFunctions();
+ }
+ }
+
+ public void GetPrimaryKeys(TGetPrimaryKeysReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ GetPrimaryKeys_call method_call = new GetPrimaryKeys_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class GetPrimaryKeys_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TGetPrimaryKeysReq req;
+ public GetPrimaryKeys_call(TGetPrimaryKeysReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetPrimaryKeys", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ GetPrimaryKeys_args args = new GetPrimaryKeys_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TGetPrimaryKeysResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_GetPrimaryKeys();
+ }
+ }
+
+ public void GetCrossReference(TGetCrossReferenceReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ GetCrossReference_call method_call = new GetCrossReference_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class GetCrossReference_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TGetCrossReferenceReq req;
+ public GetCrossReference_call(TGetCrossReferenceReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetCrossReference", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ GetCrossReference_args args = new GetCrossReference_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TGetCrossReferenceResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_GetCrossReference();
+ }
+ }
+
+ public void GetOperationStatus(TGetOperationStatusReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ GetOperationStatus_call method_call = new GetOperationStatus_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class GetOperationStatus_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TGetOperationStatusReq req;
+ public GetOperationStatus_call(TGetOperationStatusReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetOperationStatus", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ GetOperationStatus_args args = new GetOperationStatus_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TGetOperationStatusResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_GetOperationStatus();
+ }
+ }
+
+ public void CancelOperation(TCancelOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ CancelOperation_call method_call = new CancelOperation_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class CancelOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TCancelOperationReq req;
+ public CancelOperation_call(TCancelOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CancelOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ CancelOperation_args args = new CancelOperation_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TCancelOperationResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_CancelOperation();
+ }
+ }
+
+ public void CloseOperation(TCloseOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ CloseOperation_call method_call = new CloseOperation_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class CloseOperation_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TCloseOperationReq req;
+ public CloseOperation_call(TCloseOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CloseOperation", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ CloseOperation_args args = new CloseOperation_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TCloseOperationResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_CloseOperation();
+ }
+ }
+
+ public void GetResultSetMetadata(TGetResultSetMetadataReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ GetResultSetMetadata_call method_call = new GetResultSetMetadata_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class GetResultSetMetadata_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TGetResultSetMetadataReq req;
+ public GetResultSetMetadata_call(TGetResultSetMetadataReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetResultSetMetadata", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ GetResultSetMetadata_args args = new GetResultSetMetadata_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TGetResultSetMetadataResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_GetResultSetMetadata();
+ }
+ }
+
+ public void FetchResults(TFetchResultsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ FetchResults_call method_call = new FetchResults_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class FetchResults_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TFetchResultsReq req;
+ public FetchResults_call(TFetchResultsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("FetchResults", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ FetchResults_args args = new FetchResults_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TFetchResultsResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_FetchResults();
+ }
+ }
+
+ public void GetDelegationToken(TGetDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ GetDelegationToken_call method_call = new GetDelegationToken_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class GetDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TGetDelegationTokenReq req;
+ public GetDelegationToken_call(TGetDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetDelegationToken", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ GetDelegationToken_args args = new GetDelegationToken_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TGetDelegationTokenResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_GetDelegationToken();
+ }
+ }
+
+ public void CancelDelegationToken(TCancelDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ CancelDelegationToken_call method_call = new CancelDelegationToken_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class CancelDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TCancelDelegationTokenReq req;
+ public CancelDelegationToken_call(TCancelDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CancelDelegationToken", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ CancelDelegationToken_args args = new CancelDelegationToken_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TCancelDelegationTokenResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_CancelDelegationToken();
+ }
+ }
+
+ public void RenewDelegationToken(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ RenewDelegationToken_call method_call = new RenewDelegationToken_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class RenewDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TRenewDelegationTokenReq req;
+ public RenewDelegationToken_call(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("RenewDelegationToken", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ RenewDelegationToken_args args = new RenewDelegationToken_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TRenewDelegationTokenResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_RenewDelegationToken();
+ }
+ }
+
+ public void GetQueryId(TGetQueryIdReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ GetQueryId_call method_call = new GetQueryId_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class GetQueryId_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TGetQueryIdReq req;
+ public GetQueryId_call(TGetQueryIdReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetQueryId", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ GetQueryId_args args = new GetQueryId_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TGetQueryIdResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_GetQueryId();
+ }
+ }
+
+ public void SetClientInfo(TSetClientInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+ checkReady();
+ SetClientInfo_call method_call = new SetClientInfo_call(req, resultHandler, this, ___protocolFactory, ___transport);
+ this.___currentMethod = method_call;
+ ___manager.call(method_call);
+ }
+
+ public static class SetClientInfo_call extends org.apache.thrift.async.TAsyncMethodCall {
+ private TSetClientInfoReq req;
+ public SetClientInfo_call(TSetClientInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+ super(client, protocolFactory, transport, resultHandler, false);
+ this.req = req;
+ }
+
+ public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+ prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("SetClientInfo", org.apache.thrift.protocol.TMessageType.CALL, 0));
+ SetClientInfo_args args = new SetClientInfo_args();
+ args.setReq(req);
+ args.write(prot);
+ prot.writeMessageEnd();
+ }
+
+ public TSetClientInfoResp getResult() throws org.apache.thrift.TException {
+ if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+ throw new IllegalStateException("Method call not finished!");
+ }
+ org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+ org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+ return (new Client(prot)).recv_SetClientInfo();
+ }
+ }
+
+ }
+
+ public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor {
+ private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName());
+ public Processor(I iface) {
+ super(iface, getProcessMap(new HashMap>()));
+ }
+
+ protected Processor(I iface, Map> processMap) {
+ super(iface, getProcessMap(processMap));
+ }
+
+ private static Map> getProcessMap(Map> processMap) {
+ processMap.put("OpenSession", new OpenSession());
+ processMap.put("CloseSession", new CloseSession());
+ processMap.put("GetInfo", new GetInfo());
+ processMap.put("ExecuteStatement", new ExecuteStatement());
+ processMap.put("GetTypeInfo", new GetTypeInfo());
+ processMap.put("GetCatalogs", new GetCatalogs());
+ processMap.put("GetSchemas", new GetSchemas());
+ processMap.put("GetTables", new GetTables());
+ processMap.put("GetTableTypes", new GetTableTypes());
+ processMap.put("GetColumns", new GetColumns());
+ processMap.put("GetFunctions", new GetFunctions());
+ processMap.put("GetPrimaryKeys", new GetPrimaryKeys());
+ processMap.put("GetCrossReference", new GetCrossReference());
+ processMap.put("GetOperationStatus", new GetOperationStatus());
+ processMap.put("CancelOperation", new CancelOperation());
+ processMap.put("CloseOperation", new CloseOperation());
+ processMap.put("GetResultSetMetadata", new GetResultSetMetadata());
+ processMap.put("FetchResults", new FetchResults());
+ processMap.put("GetDelegationToken", new GetDelegationToken());
+ processMap.put("CancelDelegationToken", new CancelDelegationToken());
+ processMap.put("RenewDelegationToken", new RenewDelegationToken());
+ processMap.put("GetQueryId", new GetQueryId());
+ processMap.put("SetClientInfo", new SetClientInfo());
+ return processMap;
+ }
+
+ public static class OpenSession extends org.apache.thrift.ProcessFunction {
+ public OpenSession() {
+ super("OpenSession");
+ }
+
+ public OpenSession_args getEmptyArgsInstance() {
+ return new OpenSession_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public OpenSession_result getResult(I iface, OpenSession_args args) throws org.apache.thrift.TException {
+ OpenSession_result result = new OpenSession_result();
+ result.success = iface.OpenSession(args.req);
+ return result;
+ }
+ }
+
+ public static class CloseSession extends org.apache.thrift.ProcessFunction {
+ public CloseSession() {
+ super("CloseSession");
+ }
+
+ public CloseSession_args getEmptyArgsInstance() {
+ return new CloseSession_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public CloseSession_result getResult(I iface, CloseSession_args args) throws org.apache.thrift.TException {
+ CloseSession_result result = new CloseSession_result();
+ result.success = iface.CloseSession(args.req);
+ return result;
+ }
+ }
+
+ public static class GetInfo extends org.apache.thrift.ProcessFunction {
+ public GetInfo() {
+ super("GetInfo");
+ }
+
+ public GetInfo_args getEmptyArgsInstance() {
+ return new GetInfo_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public GetInfo_result getResult(I iface, GetInfo_args args) throws org.apache.thrift.TException {
+ GetInfo_result result = new GetInfo_result();
+ result.success = iface.GetInfo(args.req);
+ return result;
+ }
+ }
+
+ public static class ExecuteStatement extends org.apache.thrift.ProcessFunction {
+ public ExecuteStatement() {
+ super("ExecuteStatement");
+ }
+
+ public ExecuteStatement_args getEmptyArgsInstance() {
+ return new ExecuteStatement_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public ExecuteStatement_result getResult(I iface, ExecuteStatement_args args) throws org.apache.thrift.TException {
+ ExecuteStatement_result result = new ExecuteStatement_result();
+ result.success = iface.ExecuteStatement(args.req);
+ return result;
+ }
+ }
+
+ public static class GetTypeInfo extends org.apache.thrift.ProcessFunction {
+ public GetTypeInfo() {
+ super("GetTypeInfo");
+ }
+
+ public GetTypeInfo_args getEmptyArgsInstance() {
+ return new GetTypeInfo_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public GetTypeInfo_result getResult(I iface, GetTypeInfo_args args) throws org.apache.thrift.TException {
+ GetTypeInfo_result result = new GetTypeInfo_result();
+ result.success = iface.GetTypeInfo(args.req);
+ return result;
+ }
+ }
+
+ public static class GetCatalogs extends org.apache.thrift.ProcessFunction {
+ public GetCatalogs() {
+ super("GetCatalogs");
+ }
+
+ public GetCatalogs_args getEmptyArgsInstance() {
+ return new GetCatalogs_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public GetCatalogs_result getResult(I iface, GetCatalogs_args args) throws org.apache.thrift.TException {
+ GetCatalogs_result result = new GetCatalogs_result();
+ result.success = iface.GetCatalogs(args.req);
+ return result;
+ }
+ }
+
+ public static class GetSchemas extends org.apache.thrift.ProcessFunction {
+ public GetSchemas() {
+ super("GetSchemas");
+ }
+
+ public GetSchemas_args getEmptyArgsInstance() {
+ return new GetSchemas_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public GetSchemas_result getResult(I iface, GetSchemas_args args) throws org.apache.thrift.TException {
+ GetSchemas_result result = new GetSchemas_result();
+ result.success = iface.GetSchemas(args.req);
+ return result;
+ }
+ }
+
+ public static class GetTables extends org.apache.thrift.ProcessFunction {
+ public GetTables() {
+ super("GetTables");
+ }
+
+ public GetTables_args getEmptyArgsInstance() {
+ return new GetTables_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public GetTables_result getResult(I iface, GetTables_args args) throws org.apache.thrift.TException {
+ GetTables_result result = new GetTables_result();
+ result.success = iface.GetTables(args.req);
+ return result;
+ }
+ }
+
+ public static class GetTableTypes extends org.apache.thrift.ProcessFunction {
+ public GetTableTypes() {
+ super("GetTableTypes");
+ }
+
+ public GetTableTypes_args getEmptyArgsInstance() {
+ return new GetTableTypes_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public GetTableTypes_result getResult(I iface, GetTableTypes_args args) throws org.apache.thrift.TException {
+ GetTableTypes_result result = new GetTableTypes_result();
+ result.success = iface.GetTableTypes(args.req);
+ return result;
+ }
+ }
+
+ public static class GetColumns extends org.apache.thrift.ProcessFunction {
+ public GetColumns() {
+ super("GetColumns");
+ }
+
+ public GetColumns_args getEmptyArgsInstance() {
+ return new GetColumns_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public GetColumns_result getResult(I iface, GetColumns_args args) throws org.apache.thrift.TException {
+ GetColumns_result result = new GetColumns_result();
+ result.success = iface.GetColumns(args.req);
+ return result;
+ }
+ }
+
+ public static class GetFunctions extends org.apache.thrift.ProcessFunction {
+ public GetFunctions() {
+ super("GetFunctions");
+ }
+
+ public GetFunctions_args getEmptyArgsInstance() {
+ return new GetFunctions_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public GetFunctions_result getResult(I iface, GetFunctions_args args) throws org.apache.thrift.TException {
+ GetFunctions_result result = new GetFunctions_result();
+ result.success = iface.GetFunctions(args.req);
+ return result;
+ }
+ }
+
+ public static class GetPrimaryKeys extends org.apache.thrift.ProcessFunction {
+ public GetPrimaryKeys() {
+ super("GetPrimaryKeys");
+ }
+
+ public GetPrimaryKeys_args getEmptyArgsInstance() {
+ return new GetPrimaryKeys_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public GetPrimaryKeys_result getResult(I iface, GetPrimaryKeys_args args) throws org.apache.thrift.TException {
+ GetPrimaryKeys_result result = new GetPrimaryKeys_result();
+ result.success = iface.GetPrimaryKeys(args.req);
+ return result;
+ }
+ }
+
+ public static class GetCrossReference extends org.apache.thrift.ProcessFunction {
+ public GetCrossReference() {
+ super("GetCrossReference");
+ }
+
+ public GetCrossReference_args getEmptyArgsInstance() {
+ return new GetCrossReference_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public GetCrossReference_result getResult(I iface, GetCrossReference_args args) throws org.apache.thrift.TException {
+ GetCrossReference_result result = new GetCrossReference_result();
+ result.success = iface.GetCrossReference(args.req);
+ return result;
+ }
+ }
+
+ public static class GetOperationStatus extends org.apache.thrift.ProcessFunction {
+ public GetOperationStatus() {
+ super("GetOperationStatus");
+ }
+
+ public GetOperationStatus_args getEmptyArgsInstance() {
+ return new GetOperationStatus_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public GetOperationStatus_result getResult(I iface, GetOperationStatus_args args) throws org.apache.thrift.TException {
+ GetOperationStatus_result result = new GetOperationStatus_result();
+ result.success = iface.GetOperationStatus(args.req);
+ return result;
+ }
+ }
+
+ public static class CancelOperation extends org.apache.thrift.ProcessFunction {
+ public CancelOperation() {
+ super("CancelOperation");
+ }
+
+ public CancelOperation_args getEmptyArgsInstance() {
+ return new CancelOperation_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public CancelOperation_result getResult(I iface, CancelOperation_args args) throws org.apache.thrift.TException {
+ CancelOperation_result result = new CancelOperation_result();
+ result.success = iface.CancelOperation(args.req);
+ return result;
+ }
+ }
+
+ public static class CloseOperation extends org.apache.thrift.ProcessFunction {
+ public CloseOperation() {
+ super("CloseOperation");
+ }
+
+ public CloseOperation_args getEmptyArgsInstance() {
+ return new CloseOperation_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public CloseOperation_result getResult(I iface, CloseOperation_args args) throws org.apache.thrift.TException {
+ CloseOperation_result result = new CloseOperation_result();
+ result.success = iface.CloseOperation(args.req);
+ return result;
+ }
+ }
+
+ public static class GetResultSetMetadata extends org.apache.thrift.ProcessFunction {
+ public GetResultSetMetadata() {
+ super("GetResultSetMetadata");
+ }
+
+ public GetResultSetMetadata_args getEmptyArgsInstance() {
+ return new GetResultSetMetadata_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public GetResultSetMetadata_result getResult(I iface, GetResultSetMetadata_args args) throws org.apache.thrift.TException {
+ GetResultSetMetadata_result result = new GetResultSetMetadata_result();
+ result.success = iface.GetResultSetMetadata(args.req);
+ return result;
+ }
+ }
+
+ public static class FetchResults extends org.apache.thrift.ProcessFunction {
+ public FetchResults() {
+ super("FetchResults");
+ }
+
+ public FetchResults_args getEmptyArgsInstance() {
+ return new FetchResults_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public FetchResults_result getResult(I iface, FetchResults_args args) throws org.apache.thrift.TException {
+ FetchResults_result result = new FetchResults_result();
+ result.success = iface.FetchResults(args.req);
+ return result;
+ }
+ }
+
+ public static class GetDelegationToken extends org.apache.thrift.ProcessFunction {
+ public GetDelegationToken() {
+ super("GetDelegationToken");
+ }
+
+ public GetDelegationToken_args getEmptyArgsInstance() {
+ return new GetDelegationToken_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public GetDelegationToken_result getResult(I iface, GetDelegationToken_args args) throws org.apache.thrift.TException {
+ GetDelegationToken_result result = new GetDelegationToken_result();
+ result.success = iface.GetDelegationToken(args.req);
+ return result;
+ }
+ }
+
+ public static class CancelDelegationToken extends org.apache.thrift.ProcessFunction {
+ public CancelDelegationToken() {
+ super("CancelDelegationToken");
+ }
+
+ public CancelDelegationToken_args getEmptyArgsInstance() {
+ return new CancelDelegationToken_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public CancelDelegationToken_result getResult(I iface, CancelDelegationToken_args args) throws org.apache.thrift.TException {
+ CancelDelegationToken_result result = new CancelDelegationToken_result();
+ result.success = iface.CancelDelegationToken(args.req);
+ return result;
+ }
+ }
+
+ public static class RenewDelegationToken extends org.apache.thrift.ProcessFunction {
+ public RenewDelegationToken() {
+ super("RenewDelegationToken");
+ }
+
+ public RenewDelegationToken_args getEmptyArgsInstance() {
+ return new RenewDelegationToken_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public RenewDelegationToken_result getResult(I iface, RenewDelegationToken_args args) throws org.apache.thrift.TException {
+ RenewDelegationToken_result result = new RenewDelegationToken_result();
+ result.success = iface.RenewDelegationToken(args.req);
+ return result;
+ }
+ }
+
+ public static class GetQueryId extends org.apache.thrift.ProcessFunction {
+ public GetQueryId() {
+ super("GetQueryId");
+ }
+
+ public GetQueryId_args getEmptyArgsInstance() {
+ return new GetQueryId_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public GetQueryId_result getResult(I iface, GetQueryId_args args) throws org.apache.thrift.TException {
+ GetQueryId_result result = new GetQueryId_result();
+ result.success = iface.GetQueryId(args.req);
+ return result;
+ }
+ }
+
+ public static class SetClientInfo extends org.apache.thrift.ProcessFunction {
+ public SetClientInfo() {
+ super("SetClientInfo");
+ }
+
+ public SetClientInfo_args getEmptyArgsInstance() {
+ return new SetClientInfo_args();
+ }
+
+ protected boolean isOneway() {
+ return false;
+ }
+
+ public SetClientInfo_result getResult(I iface, SetClientInfo_args args) throws org.apache.thrift.TException {
+ SetClientInfo_result result = new SetClientInfo_result();
+ result.success = iface.SetClientInfo(args.req);
+ return result;
+ }
+ }
+
+ }
+
+ public static class AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor {
+ private static final Logger LOGGER = LoggerFactory.getLogger(AsyncProcessor.class.getName());
+ public AsyncProcessor(I iface) {
+ super(iface, getProcessMap(new HashMap>()));
+ }
+
+ protected AsyncProcessor(I iface, Map> processMap) {
+ super(iface, getProcessMap(processMap));
+ }
+
+ private static Map> getProcessMap(Map> processMap) {
+ processMap.put("OpenSession", new OpenSession());
+ processMap.put("CloseSession", new CloseSession());
+ processMap.put("GetInfo", new GetInfo());
+ processMap.put("ExecuteStatement", new ExecuteStatement());
+ processMap.put("GetTypeInfo", new GetTypeInfo());
+ processMap.put("GetCatalogs", new GetCatalogs());
+ processMap.put("GetSchemas", new GetSchemas());
+ processMap.put("GetTables", new GetTables());
+ processMap.put("GetTableTypes", new GetTableTypes());
+ processMap.put("GetColumns", new GetColumns());
+ processMap.put("GetFunctions", new GetFunctions());
+ processMap.put("GetPrimaryKeys", new GetPrimaryKeys());
+ processMap.put("GetCrossReference", new GetCrossReference());
+ processMap.put("GetOperationStatus", new GetOperationStatus());
+ processMap.put("CancelOperation", new CancelOperation());
+ processMap.put("CloseOperation", new CloseOperation());
+ processMap.put("GetResultSetMetadata", new GetResultSetMetadata());
+ processMap.put("FetchResults", new FetchResults());
+ processMap.put("GetDelegationToken", new GetDelegationToken());
+ processMap.put("CancelDelegationToken", new CancelDelegationToken());
+ processMap.put("RenewDelegationToken", new RenewDelegationToken());
+ processMap.put("GetQueryId", new GetQueryId());
+ processMap.put("SetClientInfo", new SetClientInfo());
+ return processMap;
+ }
+
+ public static class OpenSession extends org.apache.thrift.AsyncProcessFunction {
+ public OpenSession() {
+ super("OpenSession");
+ }
+
+ public OpenSession_args getEmptyArgsInstance() {
+ return new OpenSession_args();
+ }
+
+ public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new AsyncMethodCallback() {
+ public void onComplete(TOpenSessionResp o) {
+ OpenSession_result result = new OpenSession_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;
+ OpenSession_result result = new OpenSession_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, OpenSession_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException {
+ iface.OpenSession(args.req,resultHandler);
+ }
+ }
+
+ public static class CloseSession extends org.apache.thrift.AsyncProcessFunction {
+ public CloseSession() {
+ super("CloseSession");
+ }
+
+ public CloseSession_args getEmptyArgsInstance() {
+ return new CloseSession_args();
+ }
+
+ public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new AsyncMethodCallback() {
+ public void onComplete(TCloseSessionResp o) {
+ CloseSession_result result = new CloseSession_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;
+ CloseSession_result result = new CloseSession_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, CloseSession_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException {
+ iface.CloseSession(args.req,resultHandler);
+ }
+ }
+
+ public static class GetInfo extends org.apache.thrift.AsyncProcessFunction {
+ public GetInfo() {
+ super("GetInfo");
+ }
+
+ public GetInfo_args getEmptyArgsInstance() {
+ return new GetInfo_args();
+ }
+
+ public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new AsyncMethodCallback() {
+ public void onComplete(TGetInfoResp o) {
+ GetInfo_result result = new GetInfo_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;
+ GetInfo_result result = new GetInfo_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, GetInfo_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException {
+ iface.GetInfo(args.req,resultHandler);
+ }
+ }
+
+ public static class ExecuteStatement extends org.apache.thrift.AsyncProcessFunction {
+ public ExecuteStatement() {
+ super("ExecuteStatement");
+ }
+
+ public ExecuteStatement_args getEmptyArgsInstance() {
+ return new ExecuteStatement_args();
+ }
+
+ public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new AsyncMethodCallback() {
+ public void onComplete(TExecuteStatementResp o) {
+ ExecuteStatement_result result = new ExecuteStatement_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;
+ ExecuteStatement_result result = new ExecuteStatement_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, ExecuteStatement_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException {
+ iface.ExecuteStatement(args.req,resultHandler);
+ }
+ }
+
+ public static class GetTypeInfo extends org.apache.thrift.AsyncProcessFunction {
+ public GetTypeInfo() {
+ super("GetTypeInfo");
+ }
+
+ public GetTypeInfo_args getEmptyArgsInstance() {
+ return new GetTypeInfo_args();
+ }
+
+ public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new AsyncMethodCallback() {
+ public void onComplete(TGetTypeInfoResp o) {
+ GetTypeInfo_result result = new GetTypeInfo_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;
+ GetTypeInfo_result result = new GetTypeInfo_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, GetTypeInfo_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException {
+ iface.GetTypeInfo(args.req,resultHandler);
+ }
+ }
+
+ public static class GetCatalogs extends org.apache.thrift.AsyncProcessFunction {
+ public GetCatalogs() {
+ super("GetCatalogs");
+ }
+
+ public GetCatalogs_args getEmptyArgsInstance() {
+ return new GetCatalogs_args();
+ }
+
+ public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new AsyncMethodCallback() {
+ public void onComplete(TGetCatalogsResp o) {
+ GetCatalogs_result result = new GetCatalogs_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;
+ GetCatalogs_result result = new GetCatalogs_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, GetCatalogs_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException {
+ iface.GetCatalogs(args.req,resultHandler);
+ }
+ }
+
+ public static class GetSchemas extends org.apache.thrift.AsyncProcessFunction {
+ public GetSchemas() {
+ super("GetSchemas");
+ }
+
+ public GetSchemas_args getEmptyArgsInstance() {
+ return new GetSchemas_args();
+ }
+
+ public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new AsyncMethodCallback() {
+ public void onComplete(TGetSchemasResp o) {
+ GetSchemas_result result = new GetSchemas_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;
+ GetSchemas_result result = new GetSchemas_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, GetSchemas_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException {
+ iface.GetSchemas(args.req,resultHandler);
+ }
+ }
+
+ public static class GetTables extends org.apache.thrift.AsyncProcessFunction {
+ public GetTables() {
+ super("GetTables");
+ }
+
+ public GetTables_args getEmptyArgsInstance() {
+ return new GetTables_args();
+ }
+
+ public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new AsyncMethodCallback() {
+ public void onComplete(TGetTablesResp o) {
+ GetTables_result result = new GetTables_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;
+ GetTables_result result = new GetTables_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, GetTables_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException {
+ iface.GetTables(args.req,resultHandler);
+ }
+ }
+
+ public static class GetTableTypes extends org.apache.thrift.AsyncProcessFunction {
+ public GetTableTypes() {
+ super("GetTableTypes");
+ }
+
+ public GetTableTypes_args getEmptyArgsInstance() {
+ return new GetTableTypes_args();
+ }
+
+ public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new AsyncMethodCallback() {
+ public void onComplete(TGetTableTypesResp o) {
+ GetTableTypes_result result = new GetTableTypes_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;
+ GetTableTypes_result result = new GetTableTypes_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, GetTableTypes_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException {
+ iface.GetTableTypes(args.req,resultHandler);
+ }
+ }
+
+ public static class GetColumns extends org.apache.thrift.AsyncProcessFunction {
+ public GetColumns() {
+ super("GetColumns");
+ }
+
+ public GetColumns_args getEmptyArgsInstance() {
+ return new GetColumns_args();
+ }
+
+ public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new AsyncMethodCallback() {
+ public void onComplete(TGetColumnsResp o) {
+ GetColumns_result result = new GetColumns_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;
+ GetColumns_result result = new GetColumns_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, GetColumns_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException {
+ iface.GetColumns(args.req,resultHandler);
+ }
+ }
+
+ public static class GetFunctions extends org.apache.thrift.AsyncProcessFunction {
+ public GetFunctions() {
+ super("GetFunctions");
+ }
+
+ public GetFunctions_args getEmptyArgsInstance() {
+ return new GetFunctions_args();
+ }
+
+ public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new AsyncMethodCallback() {
+ public void onComplete(TGetFunctionsResp o) {
+ GetFunctions_result result = new GetFunctions_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;
+ GetFunctions_result result = new GetFunctions_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, GetFunctions_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException {
+ iface.GetFunctions(args.req,resultHandler);
+ }
+ }
+
+ public static class GetPrimaryKeys extends org.apache.thrift.AsyncProcessFunction {
+ public GetPrimaryKeys() {
+ super("GetPrimaryKeys");
+ }
+
+ public GetPrimaryKeys_args getEmptyArgsInstance() {
+ return new GetPrimaryKeys_args();
+ }
+
+ public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new AsyncMethodCallback() {
+ public void onComplete(TGetPrimaryKeysResp o) {
+ GetPrimaryKeys_result result = new GetPrimaryKeys_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;
+ GetPrimaryKeys_result result = new GetPrimaryKeys_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, GetPrimaryKeys_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException {
+ iface.GetPrimaryKeys(args.req,resultHandler);
+ }
+ }
+
+ public static class GetCrossReference extends org.apache.thrift.AsyncProcessFunction {
+ public GetCrossReference() {
+ super("GetCrossReference");
+ }
+
+ public GetCrossReference_args getEmptyArgsInstance() {
+ return new GetCrossReference_args();
+ }
+
+ public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) {
+ final org.apache.thrift.AsyncProcessFunction fcall = this;
+ return new AsyncMethodCallback() {
+ public void onComplete(TGetCrossReferenceResp o) {
+ GetCrossReference_result result = new GetCrossReference_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;
+ GetCrossReference_result result = new GetCrossReference_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, GetCrossReference_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException {
+ iface.GetCrossReference(args.req,resultHandler);
+ }
+ }
+
+ public static class GetOperationStatus extends org.apache.thrift.AsyncProcessFunction {
+ public GetOperationStatus() {
+ super("GetOperationStatus");
+ }
+
+ public GetOperationStatus_args getEmptyArgsInstance() {
+ return new GetOperationStatus_args();
+ }
+
+ public AsyncMethodCallback