You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2013/03/12 19:22:03 UTC

svn commit: r1455659 [7/11] - in /hive/trunk: ./ ant/src/org/apache/hadoop/hive/ant/ bin/ bin/ext/ cli/ common/ common/src/gen/ common/src/gen/org/ common/src/gen/org/apache/ common/src/gen/org/apache/hive/ common/src/gen/org/apache/hive/common/ common...

Modified: hive/trunk/ql/build.xml
URL: http://svn.apache.org/viewvc/hive/trunk/ql/build.xml?rev=1455659&r1=1455658&r2=1455659&view=diff
==============================================================================
--- hive/trunk/ql/build.xml (original)
+++ hive/trunk/ql/build.xml Tue Mar 12 18:22:00 2013
@@ -30,26 +30,22 @@
   <property name="ql.test.results.dir" location="${basedir}/src/test/results"/>
 
   <property name="ql.test.query.clientpositive.dir" location="${ql.test.query.dir}/clientpositive"/>
+  <property name="ql.test.query.beelinepositive.dir" location="${ql.test.query.dir}/beelinepositive"/>
   <property name="ql.test.results.clientpositive.dir" location="${ql.test.results.dir}/clientpositive"/>
+  <property name="ql.test.results.beelinepositive.dir" location="${ql.test.results.dir}/beelinepositive"/>
 
   <import file="../build-common.xml"/>
   <property name="ql.test.clientpositive.exclude" value="${minimr.query.files}"/>
 
-  <target name="thriftif" depends="check-thrift-home">
-    <echo message="${ant.project.name}"/>
-    <fail unless="thrift.home">You must set the 'thrift.home' property!</fail>
-    <echo>Executing ${thrift.home}/bin/thrift on ${ant.project.name}/if/queryplan.thrift</echo>
-    <exec executable="${thrift.home}/bin/thrift"  failonerror="true" dir=".">
-      <arg line="${thrift.args} -I ${basedir}/include -I ${basedir}/.. -o ${basedir}/src/gen/thrift if/queryplan.thrift " />
-    </exec>
-  </target>
-
+  <property name="ql.test.beelinepositive.exclude" value="${test.beelinepositive.exclude}"/>
+  
   <target name="gen-test" depends="test-conditions, test-init" >
     <echo message="${ant.project.name}"/>
     <taskdef name="qtestgen" classname="org.apache.hadoop.hive.ant.QTestGenTask"
              classpath="${build.dir.hive}/anttasks/hive-anttasks-${version}.jar:${build.ivy.lib.dir}/default/velocity-${velocity.version}.jar:${build.ivy.lib.dir}/default/commons-collections-${commons-collections.version}.jar:${build.ivy.lib.dir}/default/commons-lang-${commons-lang.version}.jar:${build.ivy.lib.dir}/default/derby-${derby.version}.jar"/>
     
-    <qtestgen outputDirectory="${test.build.src}/org/apache/hadoop/hive/ql/parse" 
+    <qtestgen hiveRootDirectory="${hive.root}"
+              outputDirectory="${test.build.src}/org/apache/hadoop/hive/ql/parse" 
               templatePath="${ql.test.template.dir}" template="TestParse.vm" 
               queryDirectory="${ql.test.query.dir}/positive"
               queryFile="${qfile}"
@@ -60,7 +56,8 @@
               hadoopVersion="${hadoopVersion}"
               logDirectory="${test.log.dir}/positive"/>
     
-    <qtestgen outputDirectory="${test.build.src}/org/apache/hadoop/hive/ql/parse" 
+    <qtestgen hiveRootDirectory="${hive.root}"
+              outputDirectory="${test.build.src}/org/apache/hadoop/hive/ql/parse" 
               templatePath="${ql.test.template.dir}" template="TestParseNegative.vm" 
               queryDirectory="${ql.test.query.dir}/negative" 
               queryFile="${qfile}"
@@ -71,7 +68,8 @@
               hadoopVersion="${hadoopVersion}"
               logDirectory="${test.log.dir}/negative"/>
 
-    <qtestgen outputDirectory="${test.build.src}/org/apache/hadoop/hive/cli" 
+    <qtestgen hiveRootDirectory="${hive.root}"
+              outputDirectory="${test.build.src}/org/apache/hadoop/hive/cli" 
               templatePath="${ql.test.template.dir}" template="TestCliDriver.vm" 
               queryDirectory="${ql.test.query.clientpositive.dir}" 
               queryFile="${qfile}"
@@ -82,15 +80,29 @@
               resultsDirectory="${ql.test.results.clientpositive.dir}" className="TestCliDriver"
               logFile="${test.log.dir}/testclidrivergen.log"
               logDirectory="${test.log.dir}/clientpositive"
-              hadoopVersion="${hadoopVersion}"
-    />
+              hadoopVersion="${hadoopVersion}"/>
+
+    <qtestgen hiveRootDirectory="${hive.root}"
+              outputDirectory="${test.build.src}/org/apache/hive/beeline/util" 
+              templatePath="${ql.test.template.dir}" template="TestBeeLineDriver.vm" 
+              queryDirectory="${ql.test.query.clientpositive.dir}" 
+              queryFile="${qfile}"
+              excludeQueryFile="${ql.test.beelinepositive.exclude}"
+              queryFileRegex="${qfile_regex}"
+              clusterMode="${clustermode}"
+              runDisabled="${run_disabled}"
+              resultsDirectory="${ql.test.results.beelinepositive.dir}" className="TestBeeLineDriver"
+              logFile="${test.log.dir}/testbeelinedrivergen.log"
+              logDirectory="${test.log.dir}/beelinepositive"
+              hadoopVersion="${hadoopVersion}" />
 
     <if>
       <not>
         <matches string="${hadoop.version.ant-internal}" pattern="^0\.17\..*" />
       </not>
       <then>
-        <qtestgen outputDirectory="${test.build.src}/org/apache/hadoop/hive/cli" 
+        <qtestgen hiveRootDirectory="${hive.root}"
+                  outputDirectory="${test.build.src}/org/apache/hadoop/hive/cli" 
                   templatePath="${ql.test.template.dir}" template="TestCliDriver.vm" 
                   queryDirectory="${ql.test.query.clientpositive.dir}" 
                   queryFile="${qfile}"
@@ -104,7 +116,8 @@
                   hadoopVersion="${hadoopVersion}"
                   />
 
-        <qtestgen outputDirectory="${test.build.src}/org/apache/hadoop/hive/cli" 
+        <qtestgen hiveRootDirectory="${hive.root}"
+                  outputDirectory="${test.build.src}/org/apache/hadoop/hive/cli" 
                   templatePath="${ql.test.template.dir}" template="TestNegativeCliDriver.vm" 
                   queryDirectory="${ql.test.query.dir}/clientnegative"
                   queryFile="${qfile}"
@@ -120,7 +133,8 @@
       </then>
     </if>
 
-    <qtestgen outputDirectory="${test.build.src}/org/apache/hadoop/hive/cli" 
+    <qtestgen hiveRootDirectory="${hive.root}"
+              outputDirectory="${test.build.src}/org/apache/hadoop/hive/cli" 
               templatePath="${ql.test.template.dir}" template="TestNegativeCliDriver.vm" 
               queryDirectory="${ql.test.query.dir}/clientnegative" 
               queryFile="${qfile}"

Modified: hive/trunk/ql/ivy.xml
URL: http://svn.apache.org/viewvc/hive/trunk/ql/ivy.xml?rev=1455659&r1=1455658&r2=1455659&view=diff
==============================================================================
--- hive/trunk/ql/ivy.xml (original)
+++ hive/trunk/ql/ivy.xml Tue Mar 12 18:22:00 2013
@@ -34,6 +34,8 @@
                 conf="test->default" transitive="false"/>
     <dependency org="org.apache.hive" name="hive-contrib" rev="${version}"
                 conf="test->default" transitive="false"/>
+    <dependency org="org.apache.hive" name="hive-testutils" rev="${version}"
+                conf="test->default" transitive="false"/>
     <dependency org="com.google.protobuf" name="protobuf-java" 
                 rev="${protobuf.version}" transitive="false"/>
     <dependency org="org.iq80.snappy" name="snappy" 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java?rev=1455659&r1=1455658&r2=1455659&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java Tue Mar 12 18:22:00 2013
@@ -112,6 +112,8 @@ public class Driver implements CommandPr
   static final private Log LOG = LogFactory.getLog(Driver.class.getName());
   static final private LogHelper console = new LogHelper(LOG);
 
+  private static final Object compileMonitor = new Object();
+  
   private int maxRows = 100;
   ByteStream.Output bos = new ByteStream.Output();
 
@@ -895,7 +897,10 @@ public class Driver implements CommandPr
     perfLogger.PerfLogBegin(LOG, PerfLogger.DRIVER_RUN);
     perfLogger.PerfLogBegin(LOG, PerfLogger.TIME_TO_SUBMIT);
 
-    int ret = compile(command);
+    int ret;
+    synchronized (compileMonitor) {
+      ret = compile(command);
+    }
     if (ret != 0) {
       releaseLocks(ctx.getHiveLocks());
       return new CommandProcessorResponse(ret, errorMessage, SQLState);

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/CopyTask.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/CopyTask.java?rev=1455659&r1=1455658&r2=1455659&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/CopyTask.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/CopyTask.java Tue Mar 12 18:22:00 2013
@@ -72,13 +72,12 @@ public class CopyTask extends Task<CopyW
       }
 
       if (!dstFs.mkdirs(toPath)) {
-        console
-            .printError("Cannot make target directory: " + toPath.toString());
+        console.printError("Cannot make target directory: " + toPath.toString());
         return 2;
       }
 
       for (FileStatus oneSrc : srcs) {
-        System.out.println("Copying file: " + oneSrc.getPath().toString());
+        console.printInfo("Copying file: " + oneSrc.getPath().toString());
         LOG.debug("Copying file: " + oneSrc.getPath().toString());
         if (!FileUtil.copy(srcFs, oneSrc.getPath(), dstFs, toPath, false, // delete
             // source

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java?rev=1455659&r1=1455658&r2=1455659&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java Tue Mar 12 18:22:00 2013
@@ -182,6 +182,10 @@ public class Hive {
     return db;
   }
 
+  public static void set(Hive hive) {
+    hiveDB.set(hive);
+  }
+
   public static void closeCurrent() {
     hiveDB.remove();
   }
@@ -2409,6 +2413,26 @@ private void constructOneLBLocationMap(F
     }
   }
 
+  public String getDelegationToken(String owner, String renewer)
+    throws HiveException{
+    try {
+      return getMSC().getDelegationToken(owner, renewer);
+    } catch(Exception e) {
+      LOG.error(StringUtils.stringifyException(e));
+      throw new HiveException(e);
+    }
+  }
+
+  public void cancelDelegationToken(String tokenStrForm)
+    throws HiveException {
+    try {
+      getMSC().cancelDelegationToken(tokenStrForm);
+    }  catch(Exception e) {
+      LOG.error(StringUtils.stringifyException(e));
+      throw new HiveException(e);
+    }
+  }
+
   private static String[] getQualifiedNames(String qualifiedName) {
     return qualifiedName.split("\\.");
   }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java?rev=1455659&r1=1455658&r2=1455659&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java Tue Mar 12 18:22:00 2013
@@ -28,8 +28,8 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.Schema;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Schema;
 import org.apache.hadoop.hive.ql.parse.VariableSubstitution;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
@@ -116,20 +116,29 @@ public class SetProcessor implements Com
       return new CommandProcessorResponse(0);
     } else if (varname.startsWith(SetProcessor.HIVECONF_PREFIX)){
       String propName = varname.substring(SetProcessor.HIVECONF_PREFIX.length());
-      String error = setConf(varname, propName, varvalue, false);
-      return new CommandProcessorResponse(error == null ? 0 : 1, error, null);
+      try {
+        setConf(varname, propName, varvalue, false);
+        return new CommandProcessorResponse(0);
+      } catch (IllegalArgumentException e) {
+        return new CommandProcessorResponse(1, e.getMessage(), "42000");
+      }
     } else if (varname.startsWith(SetProcessor.HIVEVAR_PREFIX)) {
       String propName = varname.substring(SetProcessor.HIVEVAR_PREFIX.length());
       ss.getHiveVariables().put(propName, new VariableSubstitution().substitute(ss.getConf(),varvalue));
       return new CommandProcessorResponse(0);
     } else {
-      String error = setConf(varname, varname, varvalue, true);
-      return new CommandProcessorResponse(error == null ? 0 : 1, error, null);
+      try {
+        setConf(varname, varname, varvalue, true);
+        return new CommandProcessorResponse(0);
+      } catch (IllegalArgumentException e) {
+        return new CommandProcessorResponse(1, e.getMessage(), "42000");
+      }
     }
   }
 
   // returns non-null string for validation fail
-  private String setConf(String varname, String key, String varvalue, boolean register) {
+  private void setConf(String varname, String key, String varvalue, boolean register)
+        throws IllegalArgumentException {
     HiveConf conf = SessionState.get().getConf();
     String value = new VariableSubstitution().substitute(conf, varvalue);
     if (conf.getBoolVar(HiveConf.ConfVars.HIVECONFVALIDATION)) {
@@ -139,14 +148,13 @@ public class SetProcessor implements Com
         message.append("'SET ").append(varname).append('=').append(varvalue);
         message.append("' FAILED because "); message.append(key).append(" expects an ");
         message.append(confVars.typeString()).append(" value.");
-        return message.toString();
+        throw new IllegalArgumentException(message.toString());
       }
     }
-    conf.set(key, value);
+    conf.verifyAndSet(key, value);
     if (register) {
       SessionState.get().getOverriddenConfigurations().put(key, value);
     }
-    return null;
   }
 
   private SortedMap<String,String> propertiesToSortedMap(Properties p){
@@ -209,7 +217,7 @@ public class SetProcessor implements Com
       }
     } else {
       dumpOption(varname);
-      return new CommandProcessorResponse(0);
+      return new CommandProcessorResponse(0, null, null, getSchema());
     }
   }
 

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java?rev=1455659&r1=1455658&r2=1455659&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java Tue Mar 12 18:22:00 2013
@@ -175,17 +175,14 @@ public class QTestUtil {
         normalizeNames(file);
       }
     } else {
-      // System.out.println("Trying to match: " + path.getPath());
       Matcher m = reduceTok.matcher(path.getName());
       if (m.matches()) {
         String name = m.group(1) + "reduce" + m.group(3);
-        // System.out.println("Matched new name: " + name);
         path.renameTo(new File(path.getParent(), name));
       } else {
         m = mapTok.matcher(path.getName());
         if (m.matches()) {
           String name = m.group(1) + "map_" + m.group(3);
-          // System.out.println("Matched new name: " + name);
           path.renameTo(new File(path.getParent(), name));
         }
       }
@@ -196,6 +193,14 @@ public class QTestUtil {
     this(outDir, logDir, false, "0.20");
   }
 
+  public String getOutputDirectory() {
+    return outDir;
+  }
+
+  public String getLogDirectory() {
+    return logDir;
+  }
+  
   private String getHadoopMainVersion(String input) {
     if (input == null) {
       return null;

Modified: hive/trunk/serde/build.xml
URL: http://svn.apache.org/viewvc/hive/trunk/serde/build.xml?rev=1455659&r1=1455658&r2=1455659&view=diff
==============================================================================
--- hive/trunk/serde/build.xml (original)
+++ hive/trunk/serde/build.xml Tue Mar 12 18:22:00 2013
@@ -51,27 +51,6 @@
     </javac>
   </target>
 
-  <target name="thriftif" depends="check-thrift-home">
-    <echo message="Project: ${ant.project.name}"/>
-    <fail unless="thrift.home">You must set the 'thrift.home' property!</fail>
-    <echo>Executing ${thrift.home}/bin/thrift to build java serde Constants... </echo>
-    <exec executable="${thrift.home}/bin/thrift"  failonerror="true" dir=".">
-      <arg line="${thrift.args} -o ${src.dir}/gen/thrift if/serde.thrift " />
-    </exec>
-    <echo>Executing ${thrift.home}/bin/thrift to build complex.thrift test classes... </echo>
-    <exec executable="${thrift.home}/bin/thrift"  failonerror="true" dir=".">
-      <arg line="--gen java:beans -o ${src.dir}/gen/thrift if/test/complex.thrift " />
-    </exec>
-    <echo>Executing ${thrift.home}/bin/thrift to build testthrift.thrift classes... </echo>
-    <exec executable="${thrift.home}/bin/thrift"  failonerror="true" dir=".">
-      <arg line="--gen java:beans -o ${src.dir}/gen/thrift if/test/testthrift.thrift " />
-    </exec>
-    <echo>Executing ${thrift.home}/bin/thrift to build megastruct.thrift classes... </echo>
-    <exec executable="${thrift.home}/bin/thrift"  failonerror="true" dir=".">
-      <arg line="--gen java:beans -o ${src.dir}/gen/thrift if/test/megastruct.thrift " />
-    </exec>
-  </target>
-
   <target name="gen-testdata" depends="compile-test,test-jar">
     <echo message="Project: ${ant.project.name}"/>
     <echo>Generating data/files/complex.seq... </echo>

Modified: hive/trunk/service/build.xml
URL: http://svn.apache.org/viewvc/hive/trunk/service/build.xml?rev=1455659&r1=1455658&r2=1455659&view=diff
==============================================================================
--- hive/trunk/service/build.xml (original)
+++ hive/trunk/service/build.xml Tue Mar 12 18:22:00 2013
@@ -22,15 +22,6 @@
 
   <import file="../build-common.xml"/>
 
-  <target name="thriftif" depends="check-thrift-home">
-    <echo message="Project: ${ant.project.name}"/>
-    <fail unless="thrift.home">You must set the 'thrift.home' property!</fail>
-    <echo>Executing ${thrift.home}/bin/thrift on ${ant.project.name}/if/hive_service.thrift</echo>
-    <exec executable="${thrift.home}/bin/thrift"  failonerror="true" dir=".">
-      <arg line="${thrift.args} -I ${basedir}/include -I ${basedir}/.. -o ${src.dir}/gen/thrift if/hive_service.thrift " />
-    </exec>
-  </target>
-
   <target name="compile" depends="init,ivy-retrieve">
     <echo message="Project: ${ant.project.name}"/>
     <javac

Added: hive/trunk/service/if/TCLIService.thrift
URL: http://svn.apache.org/viewvc/hive/trunk/service/if/TCLIService.thrift?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/if/TCLIService.thrift (added)
+++ hive/trunk/service/if/TCLIService.thrift Tue Mar 12 18:22:00 2013
@@ -0,0 +1,997 @@
+// 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.hive.service.cli.thrift
+namespace cpp apache.hive.service.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
+}
+
+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
+}
+  
+const set<TTypeId> 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
+]
+
+const set<TTypeId> COMPLEX_TYPES = [
+  TTypeId.ARRAY_TYPE
+  TTypeId.MAP_TYPE
+  TTypeId.STRUCT_TYPE
+  TTypeId.UNION_TYPE
+  TTypeId.USER_DEFINED_TYPE
+]
+
+const set<TTypeId> COLLECTION_TYPES = [
+  TTypeId.ARRAY_TYPE
+  TTypeId.MAP_TYPE
+]
+
+const map<TTypeId,string> 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"
+}
+
+// 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<INT>":
+// TTypeDesc {
+//   types = [
+//     TTypeEntry.array_entry {
+//       object_type_ptr = 1
+//     },
+//     TTypeEntry.primitive_entry {
+//       type = INT_TYPE
+//     }
+//   ]
+// }
+//
+// "MAP<INT,STRING>":
+// 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
+
+// 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
+}
+
+// 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<string, TTypeEntryPtr> nameToTypePtr
+}
+
+// Type entry for a UNIONTYPE type.
+struct TUnionTypeEntry {
+  1: required map<string, TTypeEntryPtr> 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<TTypeEntry> 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<TColumnDesc> 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
+}
+
+union TColumn {
+  1: list<TBoolValue> boolColumn
+  2: list<TByteValue> byteColumn
+  3: list<TI16Value> i16Column
+  4: list<TI32Value> i32Column
+  5: list<TI64Value> i64Column
+  6: list<TDoubleValue> doubleColumn
+  7: list<TStringValue> stringColumn
+}
+
+// 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
+}
+
+// Represents a row in a rowset.
+struct TRow {
+  1: required list<TColumnValue> colVals
+}
+
+// Represents a rowset
+struct TRowSet {
+  // The starting row offset of this rowset.
+  1: required i64 startRowOffset
+  2: required list<TRow> rows
+  3: optional list<TColumn> columns
+}
+
+// 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<string> 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,
+}
+
+
+// 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_V1
+  
+  // 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<string, string> configuration
+}
+
+struct TOpenSessionResp {
+  1: required TStatus status
+
+  // The protocol version that the server is using.
+  2: required TProtocolVersion serverProtocolVersion = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V1
+
+  // Session Handle
+  3: optional TSessionHandle sessionHandle
+
+  // The configuration settings for this session.
+  4: optional map<string, string> configuration
+}
+
+
+// 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,
+}
+
+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 exexcute 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<string, string> confOverlay
+}
+
+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<string> 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
+}
+  
+
+// GetOperationStatus()
+//
+// Get the status of an operation running on the server.
+struct TGetOperationStatusReq {
+  // Session to run this request against
+  1: required TOperationHandle operationHandle
+}
+
+struct TGetOperationStatusResp {
+  1: required TStatus status
+  2: optional TOperationState operationState
+}
+
+
+// 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.
+  // NOT SUPPORTED
+  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 or FETCH_FIRST. 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
+}
+
+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
+}
+
+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);
+
+  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);
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/AbstractService.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/AbstractService.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/AbstractService.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/AbstractService.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,184 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.service;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * AbstractService.
+ *
+ */
+public abstract class AbstractService implements Service {
+
+  private static final Log LOG = LogFactory.getLog(AbstractService.class);
+
+  /**
+   * Service state: initially {@link STATE#NOTINITED}.
+   */
+  private STATE state = STATE.NOTINITED;
+
+  /**
+   * Service name.
+   */
+  private final String name;
+  /**
+   * Service start time. Will be zero until the service is started.
+   */
+  private long startTime;
+
+  /**
+   * The configuration. Will be null until the service is initialized.
+   */
+  private HiveConf hiveConf;
+
+  /**
+   * List of state change listeners; it is final to ensure
+   * that it will never be null.
+   */
+  private final List<ServiceStateChangeListener> listeners =
+      new ArrayList<ServiceStateChangeListener>();
+
+  /**
+   * Construct the service.
+   *
+   * @param name
+   *          service name
+   */
+  public AbstractService(String name) {
+    this.name = name;
+  }
+
+  @Override
+  public synchronized STATE getServiceState() {
+    return state;
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @throws IllegalStateException
+   *           if the current service state does not permit
+   *           this action
+   */
+  @Override
+  public synchronized void init(HiveConf hiveConf) {
+    ensureCurrentState(STATE.NOTINITED);
+    this.hiveConf = hiveConf;
+    changeState(STATE.INITED);
+    LOG.info("Service:" + getName() + " is inited.");
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @throws IllegalStateException
+   *           if the current service state does not permit
+   *           this action
+   */
+  @Override
+  public synchronized void start() {
+    startTime = System.currentTimeMillis();
+    ensureCurrentState(STATE.INITED);
+    changeState(STATE.STARTED);
+    LOG.info("Service:" + getName() + " is started.");
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * @throws IllegalStateException
+   *           if the current service state does not permit
+   *           this action
+   */
+  @Override
+  public synchronized void stop() {
+    if (state == STATE.STOPPED ||
+        state == STATE.INITED ||
+        state == STATE.NOTINITED) {
+      // already stopped, or else it was never
+      // started (eg another service failing canceled startup)
+      return;
+    }
+    ensureCurrentState(STATE.STARTED);
+    changeState(STATE.STOPPED);
+    LOG.info("Service:" + getName() + " is stopped.");
+  }
+
+  @Override
+  public synchronized void register(ServiceStateChangeListener l) {
+    listeners.add(l);
+  }
+
+  @Override
+  public synchronized void unregister(ServiceStateChangeListener l) {
+    listeners.remove(l);
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public synchronized HiveConf getHiveConf() {
+    return hiveConf;
+  }
+
+  @Override
+  public long getStartTime() {
+    return startTime;
+  }
+
+  /**
+   * Verify that that a service is in a given state.
+   *
+   * @param currentState
+   *          the desired state
+   * @throws IllegalStateException
+   *           if the service state is different from
+   *           the desired state
+   */
+  private void ensureCurrentState(STATE currentState) {
+    ServiceOperations.ensureCurrentState(state, currentState);
+  }
+
+  /**
+   * Change to a new state and notify all listeners.
+   * This is a private method that is only invoked from synchronized methods,
+   * which avoid having to clone the listener list. It does imply that
+   * the state change listener methods should be short lived, as they
+   * will delay the state transition.
+   *
+   * @param newState
+   *          new service state
+   */
+  private void changeState(STATE newState) {
+    state = newState;
+    // notify listeners
+    for (ServiceStateChangeListener l : listeners) {
+      l.stateChanged(this);
+    }
+  }
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/BreakableService.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/BreakableService.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/BreakableService.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/BreakableService.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,121 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.service;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hive.service.Service.STATE;
+
+/**
+ * This is a service that can be configured to break on any of the lifecycle
+ * events, so test the failure handling of other parts of the service
+ * infrastructure.
+ *
+ * It retains a counter to the number of times each entry point is called -
+ * these counters are incremented before the exceptions are raised and
+ * before the superclass state methods are invoked.
+ *
+ */
+public class BreakableService extends AbstractService {
+  private boolean failOnInit;
+  private boolean failOnStart;
+  private boolean failOnStop;
+  private final int[] counts = new int[4];
+
+  public BreakableService() {
+    this(false, false, false);
+  }
+
+  public BreakableService(boolean failOnInit,
+                          boolean failOnStart,
+                          boolean failOnStop) {
+    super("BreakableService");
+    this.failOnInit = failOnInit;
+    this.failOnStart = failOnStart;
+    this.failOnStop = failOnStop;
+    inc(STATE.NOTINITED);
+  }
+
+  private int convert(STATE state) {
+    switch (state) {
+      case NOTINITED: return 0;
+      case INITED:    return 1;
+      case STARTED:   return 2;
+      case STOPPED:   return 3;
+      default:        return 0;
+    }
+  }
+
+  private void inc(STATE state) {
+    int index = convert(state);
+    counts[index] ++;
+  }
+
+  public int getCount(STATE state) {
+    return counts[convert(state)];
+  }
+
+  private void maybeFail(boolean fail, String action) {
+    if (fail) {
+      throw new BrokenLifecycleEvent(action);
+    }
+  }
+
+  @Override
+  public void init(HiveConf conf) {
+    inc(STATE.INITED);
+    maybeFail(failOnInit, "init");
+    super.init(conf);
+  }
+
+  @Override
+  public void start() {
+    inc(STATE.STARTED);
+    maybeFail(failOnStart, "start");
+    super.start();
+  }
+
+  @Override
+  public void stop() {
+    inc(STATE.STOPPED);
+    maybeFail(failOnStop, "stop");
+    super.stop();
+  }
+
+  public void setFailOnInit(boolean failOnInit) {
+    this.failOnInit = failOnInit;
+  }
+
+  public void setFailOnStart(boolean failOnStart) {
+    this.failOnStart = failOnStart;
+  }
+
+  public void setFailOnStop(boolean failOnStop) {
+    this.failOnStop = failOnStop;
+  }
+
+  /**
+   * The exception explicitly raised on a failure
+   */
+  public static class BrokenLifecycleEvent extends RuntimeException {
+    BrokenLifecycleEvent(String action) {
+      super("Lifecycle Failure during " + action);
+    }
+  }
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/CompositeService.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/CompositeService.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/CompositeService.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/CompositeService.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.service;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * CompositeService.
+ *
+ */
+public class CompositeService extends AbstractService {
+
+  private static final Log LOG = LogFactory.getLog(CompositeService.class);
+
+  private final List<Service> serviceList = new ArrayList<Service>();
+
+  public CompositeService(String name) {
+    super(name);
+  }
+
+  public Collection<Service> getServices() {
+    return Collections.unmodifiableList(serviceList);
+  }
+
+  protected synchronized void addService(Service service) {
+    serviceList.add(service);
+  }
+
+  protected synchronized boolean removeService(Service service) {
+    return serviceList.remove(service);
+  }
+
+  @Override
+  public synchronized void init(HiveConf hiveConf) {
+    for (Service service : serviceList) {
+      service.init(hiveConf);
+    }
+    super.init(hiveConf);
+  }
+
+  @Override
+  public synchronized void start() {
+    int i = 0;
+    try {
+      for (int n = serviceList.size(); i < n; i++) {
+        Service service = serviceList.get(i);
+        service.start();
+      }
+      super.start();
+    } catch (Throwable e) {
+      LOG.error("Error starting services " + getName(), e);
+      // Note that the state of the failed service is still INITED and not
+      // STARTED. Even though the last service is not started completely, still
+      // call stop() on all services including failed service to make sure cleanup
+      // happens.
+      stop(i);
+      throw new ServiceException("Failed to Start " + getName(), e);
+    }
+
+  }
+
+  @Override
+  public synchronized void stop() {
+    if (this.getServiceState() == STATE.STOPPED) {
+      // The base composite-service is already stopped, don't do anything again.
+      return;
+    }
+    if (serviceList.size() > 0) {
+      stop(serviceList.size() - 1);
+    }
+    super.stop();
+  }
+
+  private synchronized void stop(int numOfServicesStarted) {
+    // stop in reserve order of start
+    for (int i = numOfServicesStarted; i >= 0; i--) {
+      Service service = serviceList.get(i);
+      try {
+        service.stop();
+      } catch (Throwable t) {
+        LOG.info("Error stopping " + service.getName(), t);
+      }
+    }
+  }
+
+  /**
+   * JVM Shutdown hook for CompositeService which will stop the given
+   * CompositeService gracefully in case of JVM shutdown.
+   */
+  public static class CompositeServiceShutdownHook implements Runnable {
+
+    private final CompositeService compositeService;
+
+    public CompositeServiceShutdownHook(CompositeService compositeService) {
+      this.compositeService = compositeService;
+    }
+
+    @Override
+    public void run() {
+      try {
+        // Stop the Composite Service
+        compositeService.stop();
+      } catch (Throwable t) {
+        LOG.info("Error stopping " + compositeService.getName(), t);
+      }
+    }
+  }
+
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/FilterService.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/FilterService.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/FilterService.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/FilterService.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.service;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * FilterService.
+ *
+ */
+public class FilterService implements Service {
+
+
+  private final Service service;
+  private final long startTime = System.currentTimeMillis();
+
+  public FilterService(Service service) {
+    this.service = service;
+  }
+
+  @Override
+  public void init(HiveConf config) {
+    service.init(config);
+  }
+
+  @Override
+  public void start() {
+    service.start();
+  }
+
+  @Override
+  public void stop() {
+    service.stop();
+  }
+
+
+  @Override
+  public void register(ServiceStateChangeListener listener) {
+    service.register(listener);
+  }
+
+  @Override
+  public void unregister(ServiceStateChangeListener listener) {
+    service.unregister(listener);
+  }
+
+  @Override
+  public String getName() {
+    return service.getName();
+  }
+
+  @Override
+  public HiveConf getHiveConf() {
+    return service.getHiveConf();
+  }
+
+  @Override
+  public STATE getServiceState() {
+    return service.getServiceState();
+  }
+
+  @Override
+  public long getStartTime() {
+    return startTime;
+  }
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/Service.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/Service.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/Service.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/Service.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.service;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * Service.
+ *
+ */
+public interface Service {
+
+  /**
+   * Service states
+   */
+  public enum STATE {
+    /** Constructed but not initialized */
+    NOTINITED,
+
+    /** Initialized but not started or stopped */
+    INITED,
+
+    /** started and not stopped */
+    STARTED,
+
+    /** stopped. No further state transitions are permitted */
+    STOPPED
+  }
+
+  /**
+   * Initialize the service.
+   *
+   * The transition must be from {@link STATE#NOTINITED} to {@link STATE#INITED} unless the
+   * operation failed and an exception was raised.
+   *
+   * @param config
+   *          the configuration of the service
+   */
+  void init(HiveConf conf);
+
+
+  /**
+   * Start the service.
+   *
+   * The transition should be from {@link STATE#INITED} to {@link STATE#STARTED} unless the
+   * operation failed and an exception was raised.
+   */
+  void start();
+
+  /**
+   * Stop the service.
+   *
+   * This operation must be designed to complete regardless of the initial state
+   * of the service, including the state of all its internal fields.
+   */
+  void stop();
+
+  /**
+   * Register an instance of the service state change events.
+   *
+   * @param listener
+   *          a new listener
+   */
+  void register(ServiceStateChangeListener listener);
+
+  /**
+   * Unregister a previously instance of the service state change events.
+   *
+   * @param listener
+   *          the listener to unregister.
+   */
+  void unregister(ServiceStateChangeListener listener);
+
+  /**
+   * Get the name of this service.
+   *
+   * @return the service name
+   */
+  String getName();
+
+  /**
+   * Get the configuration of this service.
+   * This is normally not a clone and may be manipulated, though there are no
+   * guarantees as to what the consequences of such actions may be
+   *
+   * @return the current configuration, unless a specific implementation chooses
+   *         otherwise.
+   */
+  HiveConf getHiveConf();
+
+  /**
+   * Get the current service state
+   *
+   * @return the state of the service
+   */
+  STATE getServiceState();
+
+  /**
+   * Get the service start time
+   *
+   * @return the start time of the service. This will be zero if the service
+   *         has not yet been started.
+   */
+  long getStartTime();
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/ServiceException.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/ServiceException.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/ServiceException.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/ServiceException.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.service;
+
+/**
+ * ServiceException.
+ *
+ */
+public class ServiceException extends RuntimeException {
+
+  public ServiceException(Throwable cause) {
+    super(cause);
+  }
+
+  public ServiceException(String message) {
+    super(message);
+    }
+
+  public ServiceException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/ServiceOperations.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/ServiceOperations.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/ServiceOperations.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/ServiceOperations.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.service;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * ServiceOperations.
+ *
+ */
+public final class ServiceOperations {
+  private static final Log LOG = LogFactory.getLog(AbstractService.class);
+
+  private ServiceOperations() {
+  }
+
+  /**
+   * Verify that that a service is in a given state.
+   * @param state the actual state a service is in
+   * @param expectedState the desired state
+   * @throws IllegalStateException if the service state is different from
+   * the desired state
+   */
+  public static void ensureCurrentState(Service.STATE state,
+                                        Service.STATE expectedState) {
+    if (state != expectedState) {
+      throw new IllegalStateException("For this operation, the " +
+                                          "current service state must be "
+                                          + expectedState
+                                          + " instead of " + state);
+    }
+  }
+
+  /**
+   * Initialize a service.
+   * <p/>
+   * The service state is checked <i>before</i> the operation begins.
+   * This process is <i>not</i> thread safe.
+   * @param service a service that must be in the state
+   *   {@link Service.STATE#NOTINITED}
+   * @param configuration the configuration to initialize the service with
+   * @throws RuntimeException on a state change failure
+   * @throws IllegalStateException if the service is in the wrong state
+   */
+
+  public static void init(Service service, HiveConf configuration) {
+    Service.STATE state = service.getServiceState();
+    ensureCurrentState(state, Service.STATE.NOTINITED);
+    service.init(configuration);
+  }
+
+  /**
+   * Start a service.
+   * <p/>
+   * The service state is checked <i>before</i> the operation begins.
+   * This process is <i>not</i> thread safe.
+   * @param service a service that must be in the state
+   *   {@link Service.STATE#INITED}
+   * @throws RuntimeException on a state change failure
+   * @throws IllegalStateException if the service is in the wrong state
+   */
+
+  public static void start(Service service) {
+    Service.STATE state = service.getServiceState();
+    ensureCurrentState(state, Service.STATE.INITED);
+    service.start();
+  }
+
+  /**
+   * Initialize then start a service.
+   * <p/>
+   * The service state is checked <i>before</i> the operation begins.
+   * This process is <i>not</i> thread safe.
+   * @param service a service that must be in the state
+   *   {@link Service.STATE#NOTINITED}
+   * @param configuration the configuration to initialize the service with
+   * @throws RuntimeException on a state change failure
+   * @throws IllegalStateException if the service is in the wrong state
+   */
+  public static void deploy(Service service, HiveConf configuration) {
+    init(service, configuration);
+    start(service);
+  }
+
+  /**
+   * Stop a service.
+   * <p/>Do nothing if the service is null or not
+   * in a state in which it can be/needs to be stopped.
+   * <p/>
+   * The service state is checked <i>before</i> the operation begins.
+   * This process is <i>not</i> thread safe.
+   * @param service a service or null
+   */
+  public static void stop(Service service) {
+    if (service != null) {
+      Service.STATE state = service.getServiceState();
+      if (state == Service.STATE.STARTED) {
+        service.stop();
+      }
+    }
+  }
+
+  /**
+   * Stop a service; if it is null do nothing. Exceptions are caught and
+   * logged at warn level. (but not Throwables). This operation is intended to
+   * be used in cleanup operations
+   *
+   * @param service a service; may be null
+   * @return any exception that was caught; null if none was.
+   */
+  public static Exception stopQuietly(Service service) {
+    try {
+      stop(service);
+    } catch (Exception e) {
+      LOG.warn("When stopping the service " + service.getName()
+                   + " : " + e,
+               e);
+      return e;
+    }
+    return null;
+  }
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/ServiceStateChangeListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/ServiceStateChangeListener.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/ServiceStateChangeListener.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/ServiceStateChangeListener.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.service;
+
+/**
+ * ServiceStateChangeListener.
+ *
+ */
+public interface ServiceStateChangeListener {
+
+  /**
+   * Callback to notify of a state change. The service will already
+   * have changed state before this callback is invoked.
+   *
+   * This operation is invoked on the thread that initiated the state change,
+   * while the service itself in in a sychronized section.
+   * <ol>
+   *   <li>Any long-lived operation here will prevent the service state
+   *   change from completing in a timely manner.</li>
+   *   <li>If another thread is somehow invoked from the listener, and
+   *   that thread invokes the methods of the service (including
+   *   subclass-specific methods), there is a risk of a deadlock.</li>
+   * </ol>
+   *
+   *
+   * @param service the service that has changed.
+   */
+  void stateChanged(Service service);
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/AnonymousAuthenticationProviderImpl.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.service.auth;
+
+import javax.security.sasl.AuthenticationException;
+
+public class AnonymousAuthenticationProviderImpl implements PasswdAuthenticationProvider {
+
+  @Override
+  public void Authenticate(String user, String password) throws AuthenticationException {
+    // no-op authentication
+    return;
+  }
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/AuthenticationProviderFactory.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hive.service.auth;
+
+import javax.security.sasl.AuthenticationException;
+
+public class AuthenticationProviderFactory {
+
+  public static enum AuthMethods {
+    LDAP("LDAP"),
+    CUSTOM("CUSTOM"),
+    NONE("NONE");
+
+    String authMethod;
+
+    AuthMethods(String authMethod) {
+      this.authMethod = authMethod;
+    }
+
+    public String getAuthMethod() {
+      return authMethod;
+    }
+
+    public static AuthMethods getValidAuthMethod(String authMethodStr) throws AuthenticationException {
+      for (AuthMethods auth : AuthMethods.values()) {
+        if (authMethodStr.equals(auth.getAuthMethod())) {
+          return auth;
+        }
+      }
+      throw new AuthenticationException("Not a valid authentication method");
+    }
+  }
+
+  private AuthenticationProviderFactory () {
+  }
+
+  public static PasswdAuthenticationProvider getAuthenticationProvider(AuthMethods authMethod)
+            throws AuthenticationException {
+    if (authMethod.equals(AuthMethods.LDAP)) {
+      return new LdapAuthenticationProviderImpl();
+    } else if (authMethod.equals(AuthMethods.CUSTOM)) {
+      return new CustomAuthenticationProviderImpl();
+    } else if (authMethod.equals(AuthMethods.NONE)) {
+      return new AnonymousAuthenticationProviderImpl();
+    } else {
+      throw new AuthenticationException("Unsupported authentication method");
+    }
+  }
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/CustomAuthenticationProviderImpl.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hive.service.auth;
+
+import javax.security.sasl.AuthenticationException;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.util.ReflectionUtils;
+
+public class CustomAuthenticationProviderImpl
+  implements PasswdAuthenticationProvider {
+
+  Class<? extends PasswdAuthenticationProvider> customHandlerClass;
+  PasswdAuthenticationProvider customProvider;
+
+  @SuppressWarnings("unchecked")
+  CustomAuthenticationProviderImpl () {
+    HiveConf conf = new HiveConf();
+    this.customHandlerClass = (Class<? extends PasswdAuthenticationProvider>)
+        conf.getClass(
+            HiveConf.ConfVars.HIVE_SERVER2_CUSTOM_AUTHENTICATION_CLASS.name(),
+            PasswdAuthenticationProvider.class);
+    this.customProvider =
+        ReflectionUtils.newInstance(this.customHandlerClass, conf);
+  }
+
+  @Override
+  public void Authenticate(String user, String  password)
+      throws AuthenticationException {
+    this.customProvider.Authenticate(user, password);
+  }
+
+}

Added: hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java?rev=1455659&view=auto
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java (added)
+++ hive/trunk/service/src/java/org/apache/hive/service/auth/HiveAuthFactory.java Tue Mar 12 18:22:00 2013
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hive.service.auth;
+
+import java.io.IOException;
+
+import javax.security.auth.login.LoginException;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge;
+import org.apache.hive.service.cli.thrift.ThriftCLIService;
+import org.apache.thrift.TProcessorFactory;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.thrift.transport.TTransportFactory;
+
+public class HiveAuthFactory {
+
+  public static enum AuthTypes {
+    NOSASL("NOSASL"),
+    NONE("NONE"),
+    LDAP("LDAP"),
+    KERBEROS("KERBEROS"),
+    CUSTOM("CUSTOM");
+
+    private String authType; // Auth type for SASL
+
+    AuthTypes(String authType) {
+      this.authType = authType;
+    }
+
+    public String getAuthName() {
+      return authType;
+    }
+
+  };
+
+  private HadoopThriftAuthBridge.Server saslServer = null;
+  private String authTypeStr;
+  HiveConf conf;
+
+  public HiveAuthFactory() throws TTransportException {
+    conf = new HiveConf();
+
+    authTypeStr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_AUTHENTICATION);
+    if (authTypeStr == null) {
+      authTypeStr = AuthTypes.NONE.getAuthName();
+    }
+    if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())
+        && ShimLoader.getHadoopShims().isSecureShimImpl()) {
+      saslServer = ShimLoader.getHadoopThriftAuthBridge().createServer(
+        conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB),
+        conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL)
+        );
+    }
+  }
+
+  public TTransportFactory getAuthTransFactory() throws LoginException {
+
+    TTransportFactory transportFactory;
+
+    if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) {
+      try {
+        transportFactory = saslServer.createTransportFactory();
+      } catch (TTransportException e) {
+        throw new LoginException(e.getMessage());
+      }
+    } else if (authTypeStr.equalsIgnoreCase(AuthTypes.NONE.getAuthName())) {
+      transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr);
+    } else if (authTypeStr.equalsIgnoreCase(AuthTypes.LDAP.getAuthName())) {
+      transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr);
+    } else if (authTypeStr.equalsIgnoreCase(AuthTypes.NOSASL.getAuthName())) {
+      transportFactory = new TTransportFactory();
+    } else if (authTypeStr.equalsIgnoreCase(AuthTypes.CUSTOM.getAuthName())) {
+      transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr);
+    } else {
+      throw new LoginException("Unsupported authentication type " + authTypeStr);
+    }
+    return transportFactory;
+  }
+
+  public TProcessorFactory getAuthProcFactory(ThriftCLIService service)
+      throws LoginException {
+    if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) {
+      return KerberosSaslHelper.getKerberosProcessorFactory(saslServer, service);
+    } else {
+      return PlainSaslHelper.getPlainProcessorFactory(service);
+    }
+  }
+
+  public String getRemoteUser() {
+    if (saslServer != null) {
+      return saslServer.getRemoteUser();
+    } else {
+      return null;
+    }
+  }
+
+  /* perform kerberos login using the hadoop shim API if the configuration is available */
+  public static void loginFromKeytab(HiveConf hiveConf) throws IOException {
+    String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL);
+    String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB);
+    if (principal.isEmpty() && keyTabFile.isEmpty()) {
+      // no security configuration available
+      return;
+    } else if (!principal.isEmpty() && !keyTabFile.isEmpty()) {
+      ShimLoader.getHadoopShims().loginUserFromKeytab(principal, keyTabFile);
+    } else {
+      throw new IOException ("HiveServer2 kerberos principal or keytab is not correctly configured");
+    }
+  }
+
+}