You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2013/09/27 01:12:37 UTC

svn commit: r1526744 [1/2] - in /hive/branches/branch-0.12: common/src/java/org/apache/hadoop/hive/conf/ conf/ eclipse-templates/ ivy/ jdbc/ jdbc/src/java/org/apache/hive/jdbc/ jdbc/src/test/org/apache/hive/jdbc/ ql/src/java/org/apache/hadoop/hive/ql/s...

Author: thejas
Date: Thu Sep 26 23:12:36 2013
New Revision: 1526744

URL: http://svn.apache.org/r1526744
Log:
HIVE-4763: add support for thrift over http transport in HS2 (Vaibhav Gumashta and Mike Liddell  via Thejas Nair)

Added:
    hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/HttpBasicAuthInterceptor.java
    hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/EmbeddedThriftBinaryCLIService.java
    hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java
    hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
    hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
    hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java
    hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/thrift/TestThriftBinaryCLIService.java
    hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java
    hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java
Removed:
    hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/EmbeddedThriftCLIService.java
    hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/TestEmbeddedThriftCLIService.java
    hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/thrift/TestThriftCLIService.java
Modified:
    hive/branches/branch-0.12/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/branches/branch-0.12/conf/hive-default.xml.template
    hive/branches/branch-0.12/eclipse-templates/.classpath
    hive/branches/branch-0.12/ivy/libraries.properties
    hive/branches/branch-0.12/jdbc/ivy.xml
    hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
    hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/HiveDriver.java
    hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/Utils.java
    hive/branches/branch-0.12/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java
    hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
    hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
    hive/branches/branch-0.12/service/src/java/org/apache/hive/service/server/HiveServer2.java
    hive/branches/branch-0.12/service/src/test/org/apache/hive/service/auth/TestPlainSaslHelper.java
    hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java
    hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java

Modified: hive/branches/branch-0.12/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1526744&r1=1526743&r2=1526744&view=diff
==============================================================================
--- hive/branches/branch-0.12/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/branches/branch-0.12/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Thu Sep 26 23:12:36 2013
@@ -737,6 +737,19 @@ public class HiveConf extends Configurat
     HIVE_DDL_OUTPUT_FORMAT("hive.ddl.output.format", null),
     HIVE_ENTITY_SEPARATOR("hive.entity.separator", "@"),
 
+    // binary or http
+    HIVE_SERVER2_TRANSPORT_MODE("hive.server2.transport.mode", "binary"),
+
+    // http (over thrift) transport settings
+    HIVE_SERVER2_THRIFT_HTTP_PORT("hive.server2.thrift.http.port", 10001),
+    HIVE_SERVER2_THRIFT_HTTP_PATH("hive.server2.thrift.http.path", "cliservice"),
+    HIVE_SERVER2_THRIFT_HTTP_MIN_WORKER_THREADS("hive.server2.thrift.http.min.worker.threads", 5),
+    HIVE_SERVER2_THRIFT_HTTP_MAX_WORKER_THREADS("hive.server2.thrift.http.max.worker.threads", 500),
+
+    // binary transport settings
+    HIVE_SERVER2_THRIFT_PORT("hive.server2.thrift.port", 10000),
+    HIVE_SERVER2_THRIFT_BIND_HOST("hive.server2.thrift.bind.host", ""),
+    HIVE_SERVER2_THRIFT_SASL_QOP("hive.server2.thrift.sasl.qop", "auth"),
     HIVE_SERVER2_THRIFT_MIN_WORKER_THREADS("hive.server2.thrift.min.worker.threads", 5),
     HIVE_SERVER2_THRIFT_MAX_WORKER_THREADS("hive.server2.thrift.max.worker.threads", 500),
 
@@ -746,10 +759,6 @@ public class HiveConf extends Configurat
     // Number of seconds HiveServer2 shutdown will wait for async threads to terminate
     HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT("hive.server2.async.exec.shutdown.timeout", 10),
 
-    HIVE_SERVER2_THRIFT_PORT("hive.server2.thrift.port", 10000),
-    HIVE_SERVER2_THRIFT_BIND_HOST("hive.server2.thrift.bind.host", ""),
-    HIVE_SERVER2_THRIFT_SASL_QOP("hive.server2.thrift.sasl.qop", "auth"),
-
 
     // HiveServer2 auth configuration
     HIVE_SERVER2_AUTHENTICATION("hive.server2.authentication", "NONE"),
@@ -1268,6 +1277,6 @@ public class HiveConf extends Configurat
     } else {
       return Integer.parseInt(m.group(1));
     }
-
   }
+
 }

Modified: hive/branches/branch-0.12/conf/hive-default.xml.template
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/conf/hive-default.xml.template?rev=1526744&r1=1526743&r2=1526744&view=diff
==============================================================================
--- hive/branches/branch-0.12/conf/hive-default.xml.template (original)
+++ hive/branches/branch-0.12/conf/hive-default.xml.template Thu Sep 26 23:12:36 2013
@@ -883,7 +883,36 @@
   <description>Read from a binary stream and treat each hive.binary.record.max.length bytes as a record.
   The last record before the end of stream can have less than hive.binary.record.max.length bytes</description>
 </property>
+    
+<property>
+  <name>hive.server2.transport.mode</name>
+  <value>binary</value>
+  <description>Server transport mode. "binary" or "http".</description>
+</property>    
+
+<property>
+  <name>hive.server2.thrift.http.port</name>
+  <value>10001</value>
+  <description>Port number when in HTTP mode.</description>
+</property> 
+
+<property>
+  <name>hive.server2.thrift.http.path</name>
+  <value>cliservice</value>
+  <description>Path component of URL endpoint when in HTTP mode.</description>
+</property> 
+
+<property>
+  <name>hive.server2.thrift.http.min.worker.threads</name>
+  <value>5</value>
+  <description>Minimum number of worker threads when in HTTP mode.</description>
+</property> 
 
+<property>
+  <name>hive.server2.thrift.http.max.worker.threads</name>
+  <value>500</value>
+  <description>Maximum number of worker threads when in HTTP mode.</description>
+</property> 
 
 <property>
   <name>hive.script.recordreader</name>
@@ -1761,6 +1790,8 @@
   </description>
 </property>
 
+
+
 <property>
   <name>hive.hmshandler.retry.attempts</name>
   <value>1</value>

Modified: hive/branches/branch-0.12/eclipse-templates/.classpath
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/eclipse-templates/.classpath?rev=1526744&r1=1526743&r2=1526744&view=diff
==============================================================================
--- hive/branches/branch-0.12/eclipse-templates/.classpath (original)
+++ hive/branches/branch-0.12/eclipse-templates/.classpath Thu Sep 26 23:12:36 2013
@@ -74,6 +74,8 @@
   <classpathentry kind="lib" path="build/ivy/lib/default/commons-collections-@commons-collections.version@.jar"/>
   <classpathentry kind="lib" path="build/ivy/lib/default/bonecp-@BoneCP.version@.jar"/>
   <classpathentry kind="lib" path="build/ivy/lib/default/commons-pool-@commons-pool.version@.jar"/>
+  <classpathentry kind="lib" path="build/ivy/lib/default/httpcore-@httpcore.version@.jar"/>
+  <classpathentry kind="lib" path="build/ivy/lib/default/httpclient-@httpclient.version@.jar"/>
   <classpathentry kind="lib" path="build/ivy/lib/default/slf4j-api-@slf4j-api.version@.jar"/>
   <classpathentry kind="lib" path="build/ivy/lib/default/slf4j-log4j12-@slf4j-log4j12.version@.jar"/>
   <classpathentry kind="lib" path="build/ivy/lib/default/JavaEWAH-@javaewah.version@.jar"/>

Modified: hive/branches/branch-0.12/ivy/libraries.properties
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ivy/libraries.properties?rev=1526744&r1=1526743&r2=1526744&view=diff
==============================================================================
--- hive/branches/branch-0.12/ivy/libraries.properties (original)
+++ hive/branches/branch-0.12/ivy/libraries.properties Thu Sep 26 23:12:36 2013
@@ -43,6 +43,8 @@ commons-pool.version=1.5.4
 derby.version=10.4.2.0
 guava.version=11.0.2
 hbase.version=0.94.6.1
+httpclient.version=4.2.5
+httpcore.version=4.2.4
 jackson.version=1.8.8
 javaewah.version=0.3.2
 jdo-api.version=3.0.1

Modified: hive/branches/branch-0.12/jdbc/ivy.xml
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/jdbc/ivy.xml?rev=1526744&r1=1526743&r2=1526744&view=diff
==============================================================================
--- hive/branches/branch-0.12/jdbc/ivy.xml (original)
+++ hive/branches/branch-0.12/jdbc/ivy.xml Thu Sep 26 23:12:36 2013
@@ -29,5 +29,10 @@
   <dependencies>
     <dependency org="org.apache.hive" name="hive-cli" rev="${version}"
                 conf="compile->default" />
+    <dependency org="org.apache.httpcomponents" name="httpcore"
+                rev="${httpcore.version}"/>
+    <dependency org="org.apache.httpcomponents" name="httpclient"
+                rev="${httpclient.version}"/>
+
   </dependencies>
 </ivy-module>

Modified: hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java?rev=1526744&r1=1526743&r2=1526744&view=diff
==============================================================================
--- hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java (original)
+++ hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java Thu Sep 26 23:12:36 2013
@@ -44,20 +44,23 @@ import java.util.concurrent.Executor;
 import javax.security.sasl.Sasl;
 import javax.security.sasl.SaslException;
 
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.service.auth.KerberosSaslHelper;
 import org.apache.hive.service.auth.PlainSaslHelper;
 import org.apache.hive.service.auth.SaslQOP;
-import org.apache.hive.service.cli.thrift.EmbeddedThriftCLIService;
+import org.apache.hive.service.cli.thrift.EmbeddedThriftBinaryCLIService;
 import org.apache.hive.service.cli.thrift.TCLIService;
 import org.apache.hive.service.cli.thrift.TCloseSessionReq;
 import org.apache.hive.service.cli.thrift.TOpenSessionReq;
 import org.apache.hive.service.cli.thrift.TOpenSessionResp;
 import org.apache.hive.service.cli.thrift.TProtocolVersion;
 import org.apache.hive.service.cli.thrift.TSessionHandle;
+import org.apache.http.impl.client.DefaultHttpClient;
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.THttpClient;
 import org.apache.thrift.transport.TSocket;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
@@ -75,30 +78,49 @@ public class HiveConnection implements j
   private static final String HIVE_AUTH_PASSWD = "password";
   private static final String HIVE_ANONYMOUS_USER = "anonymous";
   private static final String HIVE_ANONYMOUS_PASSWD = "anonymous";
-
+  private final String jdbcURI;
+  private final String host;
+  private final int port;
+  private final Map<String, String> sessConfMap;
+  private final Map<String, String> hiveConfMap;
+  private final Map<String, String> hiveVarMap;
+  private final boolean isEmbeddedMode;
   private TTransport transport;
   private TCLIService.Iface client;
   private boolean isClosed = true;
   private SQLWarning warningChain = null;
   private TSessionHandle sessHandle = null;
   private final List<TProtocolVersion> supportedProtocols = new LinkedList<TProtocolVersion>();
-  /**
-   * TODO: - parse uri (use java.net.URI?).
-   */
+
   public HiveConnection(String uri, Properties info) throws SQLException {
-    Utils.JdbcConnectionParams connParams = Utils.parseURL(uri);
-    if (connParams.isEmbeddedMode()) {
-      client = new EmbeddedThriftCLIService();
+    jdbcURI = uri;
+    // parse the connection uri
+    Utils.JdbcConnectionParams connParams = Utils.parseURL(jdbcURI);
+    // extract parsed connection parameters:
+    // JDBC URL: jdbc:hive2://<host>:<port>/dbName;sess_var_list?hive_conf_list#hive_var_list
+    // each list: <key1>=<val1>;<key2>=<val2> and so on
+    // sess_var_list -> sessConfMap
+    // hive_conf_list -> hiveConfMap
+    // hive_var_list -> hiveVarMap
+    host = connParams.getHost();
+    port = connParams.getPort();
+    sessConfMap = connParams.getSessionVars();
+    hiveConfMap = connParams.getHiveConfs();
+    hiveVarMap = connParams.getHiveVars();
+    isEmbeddedMode = connParams.isEmbeddedMode();
+
+    if (isEmbeddedMode) {
+      client = new EmbeddedThriftBinaryCLIService();
     } else {
       // extract user/password from JDBC connection properties if its not supplied in the connection URL
       if (info.containsKey(HIVE_AUTH_USER)) {
-        connParams.getSessionVars().put(HIVE_AUTH_USER, info.getProperty(HIVE_AUTH_USER));
+        sessConfMap.put(HIVE_AUTH_USER, info.getProperty(HIVE_AUTH_USER));
         if (info.containsKey(HIVE_AUTH_PASSWD)) {
-            connParams.getSessionVars().put(HIVE_AUTH_PASSWD, info.getProperty(HIVE_AUTH_PASSWD));
+          sessConfMap.put(HIVE_AUTH_PASSWD, info.getProperty(HIVE_AUTH_PASSWD));
         }
       }
-
-      openTransport(uri, connParams.getHost(), connParams.getPort(), connParams.getSessionVars());
+      // open the client transport
+      openTransport();
     }
 
     // add supported protocols
@@ -107,48 +129,66 @@ public class HiveConnection implements j
     supportedProtocols.add(TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V3);
 
     // open client session
-    openSession(uri);
+    openSession();
 
-    configureConnection(connParams);
+    configureConnection();
   }
 
-  private void configureConnection(Utils.JdbcConnectionParams connParams)
-      throws SQLException {
-    // set the hive variable in session state for local mode
-    if (connParams.isEmbeddedMode()) {
-      if (!connParams.getHiveVars().isEmpty()) {
-        SessionState.get().setHiveVariables(connParams.getHiveVars());
-      }
-    } else {
-      // for remote JDBC client, try to set the conf var using 'set foo=bar'
-      Statement stmt = createStatement();
-      for (Entry<String, String> hiveConf : connParams.getHiveConfs().entrySet()) {
-        stmt.execute("set " + hiveConf.getKey() + "=" + hiveConf.getValue());
-        stmt.close();
-      }
+  private void openTransport() throws SQLException {
+    transport = isHttpTransportMode() ?
+        createHttpTransport() :
+          createBinaryTransport();
+    TProtocol protocol = new TBinaryProtocol(transport);
+    client = new TCLIService.Client(protocol);
+    try {
+      transport.open();
+    } catch (TTransportException e) {
+      throw new SQLException("Could not open connection to "
+          + jdbcURI + ": " + e.getMessage(), " 08S01", e);
+    }
+  }
 
-      // For remote JDBC client, try to set the hive var using 'set hivevar:key=value'
-      for (Entry<String, String> hiveVar : connParams.getHiveVars().entrySet()) {
-        stmt.execute("set hivevar:" + hiveVar.getKey() + "=" + hiveVar.getValue());
-        stmt.close();
-      }
+  private TTransport createHttpTransport() throws SQLException {
+    // http path should begin with "/"
+    String httpPath;
+    httpPath = hiveConfMap.get(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH.varname);
+    if(httpPath == null) {
+      httpPath = "/";
+    }
+    if(!httpPath.startsWith("/")) {
+      httpPath = "/" + httpPath;
     }
+
+    DefaultHttpClient httpClient = new DefaultHttpClient();
+    String httpUrl = hiveConfMap.get(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname) +
+        "://" + host + ":" + port + httpPath;
+    httpClient.addRequestInterceptor(
+        new HttpBasicAuthInterceptor(getUserName(), getPasswd())
+        );
+    try {
+      transport = new THttpClient(httpUrl, httpClient);
+    }
+    catch (TTransportException e) {
+      String msg =  "Could not create http connection to " +
+          jdbcURI + ". " + e.getMessage();
+      throw new SQLException(msg, " 08S01", e);
+    }
+    return transport;
   }
 
-  private void openTransport(String uri, String host, int port, Map<String, String> sessConf )
-      throws SQLException {
+  private TTransport createBinaryTransport() throws SQLException {
     transport = new TSocket(host, port);
-
     // handle secure connection if specified
-    if (!sessConf.containsKey(HIVE_AUTH_TYPE)
-        || !sessConf.get(HIVE_AUTH_TYPE).equals(HIVE_AUTH_SIMPLE)){
+    if (!sessConfMap.containsKey(HIVE_AUTH_TYPE)
+        || !sessConfMap.get(HIVE_AUTH_TYPE).equals(HIVE_AUTH_SIMPLE)) {
       try {
-        if (sessConf.containsKey(HIVE_AUTH_PRINCIPAL)) {
+        // If Kerberos
+        if (sessConfMap.containsKey(HIVE_AUTH_PRINCIPAL)) {
           Map<String, String> saslProps = new HashMap<String, String>();
           SaslQOP saslQOP = SaslQOP.AUTH;
-          if(sessConf.containsKey(HIVE_AUTH_QOP)) {
+          if(sessConfMap.containsKey(HIVE_AUTH_QOP)) {
             try {
-              saslQOP = SaslQOP.fromString(sessConf.get(HIVE_AUTH_QOP));
+              saslQOP = SaslQOP.fromString(sessConfMap.get(HIVE_AUTH_QOP));
             } catch (IllegalArgumentException e) {
               throw new SQLException("Invalid " + HIVE_AUTH_QOP + " parameter. " + e.getMessage(), "42000", e);
             }
@@ -156,35 +196,38 @@ public class HiveConnection implements j
           saslProps.put(Sasl.QOP, saslQOP.toString());
           saslProps.put(Sasl.SERVER_AUTH, "true");
           transport = KerberosSaslHelper.getKerberosTransport(
-                  sessConf.get(HIVE_AUTH_PRINCIPAL), host, transport, saslProps);
+              sessConfMap.get(HIVE_AUTH_PRINCIPAL), host, transport, saslProps);
         } else {
-          String userName = sessConf.get(HIVE_AUTH_USER);
+          String userName = sessConfMap.get(HIVE_AUTH_USER);
           if ((userName == null) || userName.isEmpty()) {
             userName = HIVE_ANONYMOUS_USER;
           }
-          String passwd = sessConf.get(HIVE_AUTH_PASSWD);
+          String passwd = sessConfMap.get(HIVE_AUTH_PASSWD);
           if ((passwd == null) || passwd.isEmpty()) {
             passwd = HIVE_ANONYMOUS_PASSWD;
           }
           transport = PlainSaslHelper.getPlainTransport(userName, passwd, transport);
         }
       } catch (SaslException e) {
-        throw new SQLException("Could not establish secure connection to "
-                  + uri + ": " + e.getMessage(), " 08S01", e);
+        throw new SQLException("Could not create secure connection to "
+            + jdbcURI + ": " + e.getMessage(), " 08S01", e);
       }
     }
+    return transport;
+  }
 
-    TProtocol protocol = new TBinaryProtocol(transport);
-    client = new TCLIService.Client(protocol);
-    try {
-      transport.open();
-    } catch (TTransportException e) {
-      throw new SQLException("Could not establish connection to "
-          + uri + ": " + e.getMessage(), " 08S01", e);
+
+  private boolean isHttpTransportMode() {
+    String transportMode =
+        hiveConfMap.get(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname);
+    if(transportMode != null && (transportMode.equalsIgnoreCase("http") ||
+        transportMode.equalsIgnoreCase("https"))) {
+      return true;
     }
+    return false;
   }
 
-  private void openSession(String uri) throws SQLException {
+  private void openSession() throws SQLException {
     TOpenSessionReq openReq = new TOpenSessionReq();
 
     // set the session configuration
@@ -201,11 +244,61 @@ public class HiveConnection implements j
       sessHandle = openResp.getSessionHandle();
     } catch (TException e) {
       throw new SQLException("Could not establish connection to "
-          + uri + ": " + e.getMessage(), " 08S01", e);
+          + jdbcURI + ": " + e.getMessage(), " 08S01", e);
     }
     isClosed = false;
   }
 
+  private void configureConnection() throws SQLException {
+    // set the hive variable in session state for local mode
+    if (isEmbeddedMode) {
+      if (!hiveVarMap.isEmpty()) {
+        SessionState.get().setHiveVariables(hiveVarMap);
+      }
+    } else {
+      // for remote JDBC client, try to set the conf var using 'set foo=bar'
+      Statement stmt = createStatement();
+      for (Entry<String, String> hiveConf : hiveConfMap.entrySet()) {
+        stmt.execute("set " + hiveConf.getKey() + "=" + hiveConf.getValue());
+      }
+
+      // For remote JDBC client, try to set the hive var using 'set hivevar:key=value'
+      for (Entry<String, String> hiveVar : hiveVarMap.entrySet()) {
+        stmt.execute("set hivevar:" + hiveVar.getKey() + "=" + hiveVar.getValue());
+      }
+      stmt.close();
+    }
+  }
+
+  /**
+   * @return username from sessConfMap
+   */
+  private String getUserName() {
+    return getSessionValue(HIVE_AUTH_USER, HIVE_ANONYMOUS_USER);
+  }
+
+  /**
+   * @return password from sessConfMap
+   */
+  private String getPasswd() {
+    return getSessionValue(HIVE_AUTH_PASSWD, HIVE_ANONYMOUS_PASSWD);
+  }
+
+  /**
+   * Lookup varName in sessConfMap, if its null or empty return the default
+   * value varDefault
+   * @param varName
+   * @param varDefault
+   * @return
+   */
+  private String getSessionValue(String varName, String varDefault) {
+    String varValue = sessConfMap.get(varName);
+    if ((varValue == null) || varValue.isEmpty()) {
+      varValue = varDefault;
+    }
+    return varValue;
+  }
+
   public void abort(Executor executor) throws SQLException {
     // JDK 1.7
     throw new SQLException("Method not supported");

Modified: hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/HiveDriver.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/HiveDriver.java?rev=1526744&r1=1526743&r2=1526744&view=diff
==============================================================================
--- hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/HiveDriver.java (original)
+++ hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/HiveDriver.java Thu Sep 26 23:12:36 2013
@@ -30,6 +30,10 @@ import java.util.jar.Attributes;
 import java.util.jar.Manifest;
 import java.util.logging.Logger;
 import java.util.regex.Pattern;
+
+import org.apache.hive.jdbc.Utils.JdbcConnectionParams;
+
+
 /**
  * HiveDriver.
  *
@@ -50,16 +54,6 @@ public class HiveDriver implements Drive
   private static final boolean JDBC_COMPLIANT = false;
 
   /**
-   * The required prefix for the connection URL.
-   */
-  private static final String URL_PREFIX = "jdbc:hive2://";
-
-  /**
-   * If host is provided, without a port.
-   */
-  private static final String DEFAULT_PORT = "10000";
-
-  /**
    * Property key for the database name.
    */
   private static final String DBNAME_PROPERTY_KEY = "DBNAME";
@@ -99,7 +93,7 @@ public class HiveDriver implements Drive
    */
 
   public boolean acceptsURL(String url) throws SQLException {
-    return Pattern.matches(URL_PREFIX + ".*", url);
+    return Pattern.matches(Utils.URL_PREFIX + ".*", url);
   }
 
   /*
@@ -183,8 +177,8 @@ public class HiveDriver implements Drive
       info = new Properties();
     }
 
-    if ((url != null) && url.startsWith(URL_PREFIX)) {
-      info = parseURL(url, info);
+    if ((url != null) && url.startsWith(Utils.URL_PREFIX)) {
+      info = parseURLforPropertyInfo(url, info);
     }
 
     DriverPropertyInfo hostProp = new DriverPropertyInfo(HOST_PROPERTY_KEY,
@@ -214,7 +208,6 @@ public class HiveDriver implements Drive
   /**
    * Returns whether the driver is JDBC compliant.
    */
-
   public boolean jdbcCompliant() {
     return JDBC_COMPLIANT;
   }
@@ -223,44 +216,36 @@ public class HiveDriver implements Drive
    * Takes a url in the form of jdbc:hive://[hostname]:[port]/[db_name] and
    * parses it. Everything after jdbc:hive// is optional.
    *
+   * The output from Utils.parseUrl() is massaged for the needs of getPropertyInfo
    * @param url
    * @param defaults
    * @return
    * @throws java.sql.SQLException
    */
-  private Properties parseURL(String url, Properties defaults) throws SQLException {
+  private Properties parseURLforPropertyInfo(String url, Properties defaults) throws SQLException {
     Properties urlProps = (defaults != null) ? new Properties(defaults)
         : new Properties();
 
-    if (url == null || !url.startsWith(URL_PREFIX)) {
+    if (url == null || !url.startsWith(Utils.URL_PREFIX)) {
       throw new SQLException("Invalid connection url: " + url);
     }
 
-    if (url.length() <= URL_PREFIX.length()) {
-      return urlProps;
-    }
-
-    // [hostname]:[port]/[db_name]
-    String connectionInfo = url.substring(URL_PREFIX.length());
-
-    // [hostname]:[port] [db_name]
-    String[] hostPortAndDatabase = connectionInfo.split("/", 2);
-
-    // [hostname]:[port]
-    if (hostPortAndDatabase[0].length() > 0) {
-      String[] hostAndPort = hostPortAndDatabase[0].split(":", 2);
-      urlProps.put(HOST_PROPERTY_KEY, hostAndPort[0]);
-      if (hostAndPort.length > 1) {
-        urlProps.put(PORT_PROPERTY_KEY, hostAndPort[1]);
-      } else {
-        urlProps.put(PORT_PROPERTY_KEY, DEFAULT_PORT);
-      }
-    }
-
-    // [db_name]
-    if (hostPortAndDatabase.length > 1) {
-      urlProps.put(DBNAME_PROPERTY_KEY, hostPortAndDatabase[1]);
-    }
+    JdbcConnectionParams params = Utils.parseURL(url);
+    String host = params.getHost();
+    if (host == null){
+      host = "";
+    }
+    String port = Integer.toString(params.getPort());
+    if(host.equals("")){
+      port = "";
+    }
+    else if(port.equals("0")){
+      port = Utils.DEFAULT_PORT;
+    }
+    String db = params.getDbName();
+    urlProps.put(HOST_PROPERTY_KEY, host);
+    urlProps.put(PORT_PROPERTY_KEY, port);
+    urlProps.put(DBNAME_PROPERTY_KEY, db);
 
     return urlProps;
   }

Added: hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/HttpBasicAuthInterceptor.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/HttpBasicAuthInterceptor.java?rev=1526744&view=auto
==============================================================================
--- hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/HttpBasicAuthInterceptor.java (added)
+++ hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/HttpBasicAuthInterceptor.java Thu Sep 26 23:12:36 2013
@@ -0,0 +1,53 @@
+/**
+ * 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.jdbc;
+
+import java.io.IOException;
+
+import org.apache.http.Header;
+import org.apache.http.HttpException;
+import org.apache.http.HttpRequest;
+import org.apache.http.HttpRequestInterceptor;
+import org.apache.http.auth.UsernamePasswordCredentials;
+import org.apache.http.impl.auth.BasicScheme;
+import org.apache.http.protocol.HttpContext;
+
+/**
+ * The class is instantiated with the username and password, it is then
+ * used to add header with these credentials to HTTP requests
+ *
+ */
+public class HttpBasicAuthInterceptor implements HttpRequestInterceptor {
+
+  Header basicAuthHeader;
+  public HttpBasicAuthInterceptor(String username, String password){
+    if(username != null){
+      UsernamePasswordCredentials creds = new UsernamePasswordCredentials(username, password);
+      basicAuthHeader = BasicScheme.authenticate(creds, "UTF-8", false);
+    }
+  }
+
+  @Override
+  public void process(HttpRequest httpRequest, HttpContext httpContext) throws HttpException, IOException {
+    if(basicAuthHeader != null){
+      httpRequest.addHeader(basicAuthHeader);
+    }
+  }
+
+}

Modified: hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/Utils.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/Utils.java?rev=1526744&r1=1526743&r2=1526744&view=diff
==============================================================================
--- hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/Utils.java (original)
+++ hive/branches/branch-0.12/jdbc/src/java/org/apache/hive/jdbc/Utils.java Thu Sep 26 23:12:36 2013
@@ -169,13 +169,16 @@ public class Utils {
 
   /**
    * Parse JDBC connection URL
-   * The new format of the URL is jdbc:hive://<host>:<port>/dbName;sess_var_list?hive_conf_list#hive_var_list
+   * The new format of the URL is jdbc:hive2://<host>:<port>/dbName;sess_var_list?hive_conf_list#hive_var_list
    * where the optional sess, conf and var lists are semicolon separated <key>=<val> pairs. As before, if the
    * host/port is not specified, it the driver runs an embedded hive.
    * examples -
-   *  jdbc:hive://ubuntu:11000/db2?hive.cli.conf.printheader=true;hive.exec.mode.local.auto.inputbytes.max=9999#stab=salesTable;icol=customerID
-   *  jdbc:hive://?hive.cli.conf.printheader=true;hive.exec.mode.local.auto.inputbytes.max=9999#stab=salesTable;icol=customerID
-   *  jdbc:hive://ubuntu:11000/db2;user=foo;password=bar
+   *  jdbc:hive2://ubuntu:11000/db2?hive.cli.conf.printheader=true;hive.exec.mode.local.auto.inputbytes.max=9999#stab=salesTable;icol=customerID
+   *  jdbc:hive2://?hive.cli.conf.printheader=true;hive.exec.mode.local.auto.inputbytes.max=9999#stab=salesTable;icol=customerID
+   *  jdbc:hive2://ubuntu:11000/db2;user=foo;password=bar
+   *
+   *  Connect to http://server:10001/hs2, with specified basicAuth credentials and initial database:
+   *     jdbc:hive2://server:10001/db;user=foo;password=bar?hive.server2.transport.mode=http;hive.server2.thrift.http.path=hs2
    *
    * Note that currently the session properties are not used.
    *
@@ -189,7 +192,8 @@ public class Utils {
       throw new IllegalArgumentException("Bad URL format");
     }
 
-    // Don't parse URL with no other configuration.
+    // For URLs with no other configuration
+    // Don't parse them, but set embedded mode as true
     if (uri.equalsIgnoreCase(URL_PREFIX)) {
       connParams.setEmbeddedMode(true);
       return connParams;
@@ -197,11 +201,11 @@ public class Utils {
 
     URI jdbcURI = URI.create(uri.substring(URI_JDBC_PREFIX.length()));
 
-    //Check to prevent unintentional use of embedded mode. A missing "/" can
+    // Check to prevent unintentional use of embedded mode. A missing "/"
     // to separate the 'path' portion of URI can result in this.
-    //The missing "/" common typo while using secure mode, eg of such url -
+    // The missing "/" common typo while using secure mode, eg of such url -
     // jdbc:hive2://localhost:10000;principal=hive/HiveServer2Host@YOUR-REALM.COM
-    if((jdbcURI.getAuthority() != null) && (jdbcURI.getHost()==null)){
+    if((jdbcURI.getAuthority() != null) && (jdbcURI.getHost()==null)) {
        throw new IllegalArgumentException("Bad URL format. Hostname not found "
            + " in authority part of the url: " + jdbcURI.getAuthority()
            + ". Are you missing a '/' after the hostname ?");
@@ -264,6 +268,4 @@ public class Utils {
 
     return connParams;
   }
-
-
 }

Modified: hive/branches/branch-0.12/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java?rev=1526744&r1=1526743&r2=1526744&view=diff
==============================================================================
--- hive/branches/branch-0.12/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java (original)
+++ hive/branches/branch-0.12/jdbc/src/test/org/apache/hive/jdbc/TestJdbcDriver2.java Thu Sep 26 23:12:36 2013
@@ -45,6 +45,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hive.common.util.HiveVersionInfo;
+import org.apache.hive.jdbc.Utils.JdbcConnectionParams;
 import org.apache.hive.service.cli.operation.ClassicTableTypeMapping;
 import org.apache.hive.service.cli.operation.ClassicTableTypeMapping.ClassicTableTypes;
 import org.apache.hive.service.cli.operation.HiveTableTypeMapping;
@@ -1356,14 +1357,21 @@ public class TestJdbcDriver2 extends Tes
 
   // [url] [host] [port] [db]
   private static final String[][] URL_PROPERTIES = new String[][] {
+    // binary mode
     {"jdbc:hive2://", "", "", "default"},
     {"jdbc:hive2://localhost:10001/default", "localhost", "10001", "default"},
     {"jdbc:hive2://localhost/notdefault", "localhost", "10000", "notdefault"},
-    {"jdbc:hive2://foo:1243", "foo", "1243", "default"}};
+    {"jdbc:hive2://foo:1243", "foo", "1243", "default"},
+
+    // http mode
+    {"jdbc:hive2://server:10002/db;user=foo;password=bar?" +
+        "hive.server2.transport.mode=http;" +
+        "hive.server2.thrift.http.path=hs2",
+        "server", "10002", "db"},
+  };
 
   public void testDriverProperties() throws SQLException {
     HiveDriver driver = new HiveDriver();
-
     for (String[] testValues : URL_PROPERTIES) {
       DriverPropertyInfo[] dpi = driver.getPropertyInfo(testValues[0], null);
       assertEquals("unexpected DriverPropertyInfo array size", 3, dpi.length);
@@ -1371,7 +1379,29 @@ public class TestJdbcDriver2 extends Tes
       assertDpi(dpi[1], "PORT", testValues[2]);
       assertDpi(dpi[2], "DBNAME", testValues[3]);
     }
+  }
 
+  private static final String[][] HTTP_URL_PROPERTIES = new String[][] {
+    {"jdbc:hive2://server:10002/db;" +
+        "user=foo;password=bar?" +
+        "hive.server2.transport.mode=http;" +
+        "hive.server2.thrift.http.path=hs2", "server", "10002", "db", "http", "hs2"},
+    {"jdbc:hive2://server:10000/testdb;" +
+        "user=foo;password=bar?" +
+        "hive.server2.transport.mode=binary;" +
+        "hive.server2.thrift.http.path=", "server", "10000", "testdb", "binary", ""},
+  };
+
+  public void testParseUrlHttpMode() throws SQLException {
+    HiveDriver driver = new HiveDriver();
+    for (String[] testValues : HTTP_URL_PROPERTIES) {
+      JdbcConnectionParams params = Utils.parseURL(testValues[0]);
+      assertEquals(params.getHost(), testValues[1]);
+      assertEquals(params.getPort(), Integer.parseInt(testValues[2]));
+      assertEquals(params.getDbName(), testValues[3]);
+      assertEquals(params.getHiveConfs().get("hive.server2.transport.mode"), testValues[4]);
+      assertEquals(params.getHiveConfs().get("hive.server2.thrift.http.path"), testValues[5]);
+    }
   }
 
   private static void assertDpi(DriverPropertyInfo dpi, String name,

Modified: hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java?rev=1526744&r1=1526743&r2=1526744&view=diff
==============================================================================
--- hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java (original)
+++ hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java Thu Sep 26 23:12:36 2013
@@ -203,7 +203,7 @@ public class SessionState {
   }
 
   private static final SimpleDateFormat DATE_FORMAT =
-    new SimpleDateFormat("yyyyMMddHHmm");
+      new SimpleDateFormat("yyyyMMddHHmm");
 
   public void setCmd(String cmdString) {
     conf.setVar(HiveConf.ConfVars.HIVEQUERYSTRING, cmdString);
@@ -448,7 +448,7 @@ public class SessionState {
     } catch (IOException e) {
       console.printError("Unable to validate " + newFile + "\nException: "
           + e.getMessage(), "\n"
-          + org.apache.hadoop.util.StringUtils.stringifyException(e));
+              + org.apache.hadoop.util.StringUtils.stringifyException(e));
       return null;
     }
   }
@@ -465,7 +465,7 @@ public class SessionState {
     } catch (Exception e) {
       console.printError("Unable to register " + newJar + "\nException: "
           + e.getMessage(), "\n"
-          + org.apache.hadoop.util.StringUtils.stringifyException(e));
+              + org.apache.hadoop.util.StringUtils.stringifyException(e));
       return false;
     }
   }
@@ -479,7 +479,7 @@ public class SessionState {
     } catch (Exception e) {
       console.printError("Unable to unregister " + jarsToUnregister
           + "\nException: " + e.getMessage(), "\n"
-          + org.apache.hadoop.util.StringUtils.stringifyException(e));
+              + org.apache.hadoop.util.StringUtils.stringifyException(e));
       return false;
     }
   }
@@ -565,7 +565,7 @@ public class SessionState {
   }
 
   private final HashMap<ResourceType, Set<String>> resource_map =
-    new HashMap<ResourceType, Set<String>>();
+      new HashMap<ResourceType, Set<String>>();
 
   public String add_resource(ResourceType t, String value) {
     // By default don't convert to unix
@@ -783,7 +783,7 @@ public class SessionState {
 
   public void close() throws IOException {
     File resourceDir =
-      new File(getConf().getVar(HiveConf.ConfVars.DOWNLOADED_RESOURCES_DIR));
+        new File(getConf().getVar(HiveConf.ConfVars.DOWNLOADED_RESOURCES_DIR));
     LOG.debug("Removing resource dir " + resourceDir);
     try {
       if (resourceDir.exists()) {

Added: hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/EmbeddedThriftBinaryCLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/EmbeddedThriftBinaryCLIService.java?rev=1526744&view=auto
==============================================================================
--- hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/EmbeddedThriftBinaryCLIService.java (added)
+++ hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/EmbeddedThriftBinaryCLIService.java Thu Sep 26 23:12:36 2013
@@ -0,0 +1,37 @@
+/**
+ * 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.cli.thrift;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hive.service.cli.CLIService;
+
+
+/**
+ * EmbeddedThriftBinaryCLIService.
+ *
+ */
+public class EmbeddedThriftBinaryCLIService extends ThriftBinaryCLIService {
+
+  public EmbeddedThriftBinaryCLIService() {
+    super(new CLIService());
+    isEmbedded = true;
+    cliService.init(new HiveConf());
+    cliService.start();
+  }
+}

Added: hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java?rev=1526744&view=auto
==============================================================================
--- hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java (added)
+++ hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java Thu Sep 26 23:12:36 2013
@@ -0,0 +1,85 @@
+/**
+ * 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.cli.thrift;
+
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hive.service.auth.HiveAuthFactory;
+import org.apache.hive.service.cli.CLIService;
+import org.apache.thrift.TProcessorFactory;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.server.TThreadPoolServer;
+import org.apache.thrift.transport.TServerSocket;
+import org.apache.thrift.transport.TTransportFactory;
+
+
+public class ThriftBinaryCLIService extends ThriftCLIService {
+
+  public ThriftBinaryCLIService(CLIService cliService) {
+    super(cliService, "ThriftBinaryCLIService");
+  }
+
+  @Override
+  public void run() {
+    try {
+      hiveAuthFactory = new HiveAuthFactory();
+      TTransportFactory  transportFactory = hiveAuthFactory.getAuthTransFactory();
+      TProcessorFactory processorFactory = hiveAuthFactory.getAuthProcFactory(this);
+
+      String portString = System.getenv("HIVE_SERVER2_THRIFT_PORT");
+      if (portString != null) {
+        portNum = Integer.valueOf(portString);
+      } else {
+        portNum = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT);
+      }
+
+      String hiveHost = System.getenv("HIVE_SERVER2_THRIFT_BIND_HOST");
+      if (hiveHost == null) {
+        hiveHost = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST);
+      }
+
+      if (hiveHost != null && !hiveHost.isEmpty()) {
+        serverAddress = new InetSocketAddress(hiveHost, portNum);
+      } else {
+        serverAddress = new  InetSocketAddress(portNum);
+      }
+
+      minWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MIN_WORKER_THREADS);
+      maxWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MAX_WORKER_THREADS);
+
+      TThreadPoolServer.Args sargs = new TThreadPoolServer.Args(new TServerSocket(serverAddress))
+      .processorFactory(processorFactory)
+      .transportFactory(transportFactory)
+      .protocolFactory(new TBinaryProtocol.Factory())
+      .minWorkerThreads(minWorkerThreads)
+      .maxWorkerThreads(maxWorkerThreads);
+
+      server = new TThreadPoolServer(sargs);
+
+      LOG.info("ThriftBinaryCLIService listening on " + serverAddress);
+
+      server.serve();
+
+    } catch (Throwable t) {
+      LOG.error("Error: ", t);
+    }
+
+  }
+}
\ No newline at end of file

Modified: hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java?rev=1526744&r1=1526743&r2=1526744&view=diff
==============================================================================
--- hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java (original)
+++ hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java Thu Sep 26 23:12:36 2013
@@ -42,45 +42,37 @@ import org.apache.hive.service.cli.RowSe
 import org.apache.hive.service.cli.SessionHandle;
 import org.apache.hive.service.cli.TableSchema;
 import org.apache.thrift.TException;
-import org.apache.thrift.TProcessorFactory;
-import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.server.TServer;
-import org.apache.thrift.server.TThreadPoolServer;
-import org.apache.thrift.transport.TServerSocket;
-import org.apache.thrift.transport.TTransportFactory;
-
 
 /**
- * CLIService.
+ * ThriftCLIService.
  *
  */
-public class ThriftCLIService extends AbstractService implements TCLIService.Iface, Runnable {
+public abstract class ThriftCLIService extends AbstractService implements TCLIService.Iface, Runnable {
 
   public static final Log LOG = LogFactory.getLog(ThriftCLIService.class.getName());
 
-
   protected CLIService cliService;
   private static final TStatus OK_STATUS = new TStatus(TStatusCode.SUCCESS_STATUS);
   private static final TStatus ERROR_STATUS = new TStatus(TStatusCode.ERROR_STATUS);
 
-  private static HiveAuthFactory hiveAuthFactory;
-
-  private int portNum;
-  private InetSocketAddress serverAddress;
-  private TServer server;
+  protected int portNum;
+  protected InetSocketAddress serverAddress;
+  protected TServer server;
+  protected org.mortbay.jetty.Server httpServer;
 
   private boolean isStarted = false;
   protected boolean isEmbedded = false;
 
-  private HiveConf hiveConf;
-
-  private int minWorkerThreads;
-  private int maxWorkerThreads;
+  protected HiveConf hiveConf;
 
+  protected int minWorkerThreads;
+  protected int maxWorkerThreads;
 
+  protected static HiveAuthFactory hiveAuthFactory;
 
-  public ThriftCLIService(CLIService cliService) {
-    super("ThriftCLIService");
+  public ThriftCLIService(CLIService cliService, String serviceName) {
+    super(serviceName);
     this.cliService = cliService;
   }
 
@@ -102,7 +94,18 @@ public class ThriftCLIService extends Ab
   @Override
   public synchronized void stop() {
     if (isStarted && !isEmbedded) {
-      server.stop();
+      if(server != null) {
+        server.stop();
+        LOG.info("Thrift server has stopped");
+      }
+      if((httpServer != null) && httpServer.isStarted()) {
+        try {
+          httpServer.stop();
+          LOG.info("Http server has stopped");
+        } catch (Exception e) {
+          LOG.error("Error stopping Http server: ", e);
+        }
+      }
       isStarted = false;
     }
     super.stop();
@@ -155,10 +158,10 @@ public class ThriftCLIService extends Ab
         // The delegation token is not applicable in the given deployment mode
       }
       sessionHandle = cliService.openSessionWithImpersonation(userName, req.getPassword(),
-            req.getConfiguration(), delegationTokenStr);
+          req.getConfiguration(), delegationTokenStr);
     } else {
       sessionHandle = cliService.openSession(userName, req.getPassword(),
-            req.getConfiguration());
+          req.getConfiguration());
     }
     return sessionHandle;
   }
@@ -203,9 +206,9 @@ public class ThriftCLIService extends Ab
       Boolean runAsync = req.isRunAsync();
       OperationHandle operationHandle = runAsync ?
           cliService.executeStatementAsync(sessionHandle, statement, confOverlay)
-              : cliService.executeStatement(sessionHandle, statement, confOverlay);
-      resp.setOperationHandle(operationHandle.toTOperationHandle());
-      resp.setStatus(OK_STATUS);
+          : cliService.executeStatement(sessionHandle, statement, confOverlay);
+          resp.setOperationHandle(operationHandle.toTOperationHandle());
+          resp.setStatus(OK_STATUS);
     } catch (Exception e) {
       e.printStackTrace();
       resp.setStatus(HiveSQLException.toTStatus(e));
@@ -394,52 +397,6 @@ public class ThriftCLIService extends Ab
     return resp;
   }
 
-
   @Override
-  public void run() {
-    try {
-      hiveAuthFactory = new HiveAuthFactory();
-      TTransportFactory  transportFactory = hiveAuthFactory.getAuthTransFactory();
-      TProcessorFactory processorFactory = hiveAuthFactory.getAuthProcFactory(this);
-
-      String portString = System.getenv("HIVE_SERVER2_THRIFT_PORT");
-      if (portString != null) {
-        portNum = Integer.valueOf(portString);
-      } else {
-        portNum = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT);
-      }
-
-      String hiveHost = System.getenv("HIVE_SERVER2_THRIFT_BIND_HOST");
-      if (hiveHost == null) {
-        hiveHost = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST);
-      }
-
-      if (hiveHost != null && !hiveHost.isEmpty()) {
-        serverAddress = new InetSocketAddress(hiveHost, portNum);
-      } else {
-        serverAddress = new  InetSocketAddress(portNum);
-      }
-
-
-      minWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MIN_WORKER_THREADS);
-      maxWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MAX_WORKER_THREADS);
-
-
-      TThreadPoolServer.Args sargs = new TThreadPoolServer.Args(new TServerSocket(serverAddress))
-      .processorFactory(processorFactory)
-      .transportFactory(transportFactory)
-      .protocolFactory(new TBinaryProtocol.Factory())
-      .minWorkerThreads(minWorkerThreads)
-      .maxWorkerThreads(maxWorkerThreads);
-
-      server = new TThreadPoolServer(sargs);
-
-      LOG.info("ThriftCLIService listening on " + serverAddress);
-
-      server.serve();
-    } catch (Throwable t) {
-      t.printStackTrace();
-    }
-  }
-
+  public abstract void run();
 }

Added: hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java?rev=1526744&view=auto
==============================================================================
--- hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java (added)
+++ hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java Thu Sep 26 23:12:36 2013
@@ -0,0 +1,147 @@
+/**
+ * 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.cli.thrift;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.util.Shell;
+import org.apache.hive.service.auth.HiveAuthFactory.AuthTypes;
+import org.apache.hive.service.cli.CLIService;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.server.TServlet;
+import org.mortbay.jetty.nio.SelectChannelConnector;
+import org.mortbay.jetty.servlet.Context;
+import org.mortbay.jetty.servlet.ServletHolder;
+import org.mortbay.thread.QueuedThreadPool;
+
+
+public class ThriftHttpCLIService extends ThriftCLIService {
+
+  public ThriftHttpCLIService(CLIService cliService) {
+    super(cliService, "ThriftHttpCLIService");
+  }
+
+  @Override
+  public void run() {
+    try {
+      // Configure Jetty to serve http requests
+      // Example of a client connection URL: http://localhost:10000/servlets/thrifths2/
+      // a gateway may cause actual target URL to differ, e.g. http://gateway:port/hive2/servlets/thrifths2/
+
+      verifyHttpConfiguration(hiveConf);
+
+      String portString = System.getenv("HIVE_SERVER2_THRIFT_HTTP_PORT");
+      if (portString != null) {
+        portNum = Integer.valueOf(portString);
+      } else {
+        portNum = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT);
+      }
+
+      minWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_MIN_WORKER_THREADS);
+      maxWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_WORKER_THREADS);
+
+      String httpPath =  hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH);
+      // The config parameter can be like "path", "/path", "/path/", "path/*", "/path1/path2/*" and so on.
+      // httpPath should end up as "/*", "/path/*" or "/path1/../pathN/*"
+      if(httpPath == null || httpPath.equals("")) {
+        httpPath = "/*";
+      }
+      else {
+        if(!httpPath.startsWith("/")) {
+          httpPath = "/" + httpPath;
+        }
+        if(httpPath.endsWith("/")) {
+          httpPath = httpPath + "*";
+        }
+        if(!httpPath.endsWith("/*")) {
+          httpPath = httpPath + "/*";
+        }
+      }
+
+      httpServer = new org.mortbay.jetty.Server();
+
+      QueuedThreadPool threadPool = new QueuedThreadPool();
+      threadPool.setMinThreads(minWorkerThreads);
+      threadPool.setMaxThreads(maxWorkerThreads);
+      httpServer.setThreadPool(threadPool);
+      SelectChannelConnector connector = new SelectChannelConnector();
+      connector.setPort(portNum);
+
+      // Linux:yes, Windows:no
+      connector.setReuseAddress(!Shell.WINDOWS);
+      httpServer.addConnector(connector);
+
+      TCLIService.Processor<ThriftCLIService> processor =
+          new TCLIService.Processor<ThriftCLIService>(new EmbeddedThriftBinaryCLIService());
+
+      TProtocolFactory protocolFactory = new TBinaryProtocol.Factory();
+      TServlet thriftHttpServlet = new ThriftHttpServlet(processor, protocolFactory);
+      final Context context = new Context(httpServer, "/", Context.SESSIONS);
+      context.addServlet(new ServletHolder(thriftHttpServlet), httpPath);
+
+      // TODO: check defaults: maxTimeout, keepalive, maxBodySize, bodyRecieveDuration, etc.
+      httpServer.start();
+      String msg = "Starting CLIService in Http mode on port " + portNum +
+          " path=" + httpPath +
+          " with " + minWorkerThreads + ".." + maxWorkerThreads + " worker threads";
+      LOG.info(msg);
+      httpServer.join();
+    } catch (Throwable t) {
+      LOG.error("Error: ", t);
+    }
+  }
+
+  /**
+   * Verify that this configuration is supported by transportMode of HTTP
+   * @param hiveConf
+   */
+  private static void verifyHttpConfiguration(HiveConf hiveConf) {
+    String authType = hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION);
+
+    // error out if KERBEROS or LDAP mode is being used, it is not supported
+    if(authType.equalsIgnoreCase(AuthTypes.KERBEROS.toString()) ||
+        authType.equalsIgnoreCase(AuthTypes.LDAP.toString()) ||
+        authType.equalsIgnoreCase(AuthTypes.CUSTOM.toString())) {
+      String msg = ConfVars.HIVE_SERVER2_AUTHENTICATION + " setting of " +
+          authType + " is currently not supported with " +
+          ConfVars.HIVE_SERVER2_TRANSPORT_MODE + " setting of http";
+      LOG.fatal(msg);
+      throw new RuntimeException(msg);
+    }
+
+    // Throw exception here
+    if(authType.equalsIgnoreCase(AuthTypes.NONE.toString())) {
+      // NONE in case of thrift mode uses SASL
+      LOG.warn(ConfVars.HIVE_SERVER2_AUTHENTICATION + " setting to " +
+          authType + ". SASL is not supported with http transportMode," +
+          " so using equivalent of " + AuthTypes.NOSASL);
+    }
+
+    // doAs is currently not supported with http
+    if(hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS)) {
+      String msg = ConfVars.HIVE_SERVER2_ENABLE_DOAS + " setting of " +
+          "true is currently not supported with " +
+          ConfVars.HIVE_SERVER2_TRANSPORT_MODE + " setting of http";
+      LOG.fatal(msg);
+      throw new RuntimeException(msg);
+    }
+  }
+
+}
\ No newline at end of file

Added: hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java?rev=1526744&view=auto
==============================================================================
--- hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java (added)
+++ hive/branches/branch-0.12/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java Thu Sep 26 23:12:36 2013
@@ -0,0 +1,88 @@
+/**
+ * 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.cli.thrift;
+
+import java.io.IOException;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.codec.binary.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.thrift.TProcessor;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.server.TServlet;
+
+public class ThriftHttpServlet extends TServlet {
+
+  private static final long serialVersionUID = 1L;
+  public static final Log LOG = LogFactory.getLog(ThriftHttpServlet.class.getName());
+
+  public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory) {
+    super(processor, protocolFactory);
+  }
+
+  @Override
+  protected void doPost(HttpServletRequest request, HttpServletResponse response)
+      throws ServletException, IOException {
+    logRequestHeader(request);
+    super.doPost(request, response);
+  }
+
+  protected void logRequestHeader(HttpServletRequest request) {
+    String authHeaderBase64 = request.getHeader("Authorization");
+    if(authHeaderBase64 == null) {
+      LOG.warn("ThriftHttpServlet: no HTTP Authorization header");
+    }
+    else {
+      if(!authHeaderBase64.startsWith("Basic")) {
+        LOG.warn("ThriftHttpServlet: HTTP Authorization header exists but is not Basic.");
+      }
+      else if(LOG.isDebugEnabled()) {
+        String authHeaderBase64_Payload = authHeaderBase64.substring("Basic ".length());
+        String authHeaderString = StringUtils.newStringUtf8(
+            Base64.decodeBase64(authHeaderBase64_Payload.getBytes()));
+        String[] creds = authHeaderString.split(":");
+        String username = null;
+        String password = null;
+
+        if(creds.length >= 1) {
+          username = creds[0];
+        }
+        if(creds.length >= 2) {
+          password = creds[1];
+        }
+        if(password == null || password.equals("null") || password.equals("")) {
+          password = "<no password>";
+        }
+        else {
+          // don't log the actual password.
+          password = "******";
+        }
+        LOG.debug("HttpServlet:  HTTP Authorization header:: username=" + username +
+            " password=" + password);
+      }
+    }
+  }
+
+}
+

Modified: hive/branches/branch-0.12/service/src/java/org/apache/hive/service/server/HiveServer2.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/service/src/java/org/apache/hive/service/server/HiveServer2.java?rev=1526744&r1=1526743&r2=1526744&view=diff
==============================================================================
--- hive/branches/branch-0.12/service/src/java/org/apache/hive/service/server/HiveServer2.java (original)
+++ hive/branches/branch-0.12/service/src/java/org/apache/hive/service/server/HiveServer2.java Thu Sep 26 23:12:36 2013
@@ -26,7 +26,9 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hive.common.util.HiveStringUtils;
 import org.apache.hive.service.CompositeService;
 import org.apache.hive.service.cli.CLIService;
+import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService;
 import org.apache.hive.service.cli.thrift.ThriftCLIService;
+import org.apache.hive.service.cli.thrift.ThriftHttpCLIService;
 
 /**
  * HiveServer2.
@@ -50,9 +52,19 @@ public class HiveServer2 extends Composi
     cliService = new CLIService();
     addService(cliService);
 
-    thriftCLIService = new ThriftCLIService(cliService);
-    addService(thriftCLIService);
+    String transportMode = System.getenv("HIVE_SERVER2_TRANSPORT_MODE");
+    if(transportMode == null) {
+      transportMode = hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE);
+    }
+    if(transportMode != null && (transportMode.equalsIgnoreCase("http") ||
+        transportMode.equalsIgnoreCase("https"))) {
+      thriftCLIService = new ThriftHttpCLIService(cliService);
+    }
+    else {
+      thriftCLIService = new ThriftBinaryCLIService(cliService);
+    }
 
+    addService(thriftCLIService);
     super.init(hiveConf);
   }
 
@@ -70,7 +82,6 @@ public class HiveServer2 extends Composi
    * @param args
    */
   public static void main(String[] args) {
-
     //NOTE: It is critical to do this here so that log4j is reinitialized
     // before any of the other core hive classes are loaded
     try {
@@ -97,3 +108,4 @@ public class HiveServer2 extends Composi
   }
 
 }
+

Modified: hive/branches/branch-0.12/service/src/test/org/apache/hive/service/auth/TestPlainSaslHelper.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/service/src/test/org/apache/hive/service/auth/TestPlainSaslHelper.java?rev=1526744&r1=1526743&r2=1526744&view=diff
==============================================================================
--- hive/branches/branch-0.12/service/src/test/org/apache/hive/service/auth/TestPlainSaslHelper.java (original)
+++ hive/branches/branch-0.12/service/src/test/org/apache/hive/service/auth/TestPlainSaslHelper.java Thu Sep 26 23:12:36 2013
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hive.service.cli.CLIService;
 import org.apache.hive.service.cli.thrift.ThriftCLIService;
+import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService;
 import org.apache.thrift.TProcessorFactory;
 
 public class TestPlainSaslHelper extends TestCase {
@@ -40,7 +41,7 @@ public class TestPlainSaslHelper extends
 
     CLIService cliService = new CLIService();
     cliService.init(hconf);
-    ThriftCLIService tcliService = new ThriftCLIService(cliService);
+    ThriftCLIService tcliService = new ThriftBinaryCLIService(cliService);
     tcliService.init(hconf);
     TProcessorFactory procFactory = PlainSaslHelper.getPlainProcessorFactory(tcliService);
     assertEquals("doAs enabled processor for unsecure mode",

Modified: hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java?rev=1526744&r1=1526743&r2=1526744&view=diff
==============================================================================
--- hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java (original)
+++ hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java Thu Sep 26 23:12:36 2013
@@ -51,7 +51,7 @@ public abstract class CLIServiceTest {
   }
 
   @Test
-  public void createSessionTest() throws Exception {
+  public void openSessionTest() throws Exception {
     SessionHandle sessionHandle = client
         .openSession("tom", "password", Collections.<String, String>emptyMap());
     assertNotNull(sessionHandle);

Added: hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java?rev=1526744&view=auto
==============================================================================
--- hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java (added)
+++ hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/TestEmbeddedThriftBinaryCLIService.java Thu Sep 26 23:12:36 2013
@@ -0,0 +1,60 @@
+/**
+ * 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.cli;
+
+import org.apache.hive.service.cli.thrift.EmbeddedThriftBinaryCLIService;
+import org.apache.hive.service.cli.thrift.ThriftCLIService;
+import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+/**
+ * TestEmbeddedThriftBinaryCLIService.
+ *
+ */
+public class TestEmbeddedThriftBinaryCLIService extends CLIServiceTest {
+
+  private static ThriftCLIService service;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    service = new EmbeddedThriftBinaryCLIService();
+    client = new ThriftCLIServiceClient(service);
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceTest#setUp()
+   */
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.hive.service.cli.CLIServiceTest#tearDown()
+   */
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    super.tearDown();
+  }
+
+}

Modified: hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java?rev=1526744&r1=1526743&r2=1526744&view=diff
==============================================================================
--- hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java (original)
+++ hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/session/TestSessionHooks.java Thu Sep 26 23:12:36 2013
@@ -27,7 +27,7 @@ import junit.framework.TestCase;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.SessionHandle;
-import org.apache.hive.service.cli.thrift.EmbeddedThriftCLIService;
+import org.apache.hive.service.cli.thrift.EmbeddedThriftBinaryCLIService;
 import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient;
 import org.junit.Before;
 import org.junit.Test;
@@ -35,7 +35,7 @@ import org.junit.Test;
 public class TestSessionHooks extends TestCase {
 
   public static final String SESSION_USER_NAME = "user1";
-  private EmbeddedThriftCLIService service;
+  private EmbeddedThriftBinaryCLIService service;
   private ThriftCLIServiceClient client;
 
   public static class SessionHookTest implements HiveSessionHook {
@@ -58,7 +58,7 @@ public class TestSessionHooks extends Te
     super.setUp();
     System.setProperty(ConfVars.HIVE_SERVER2_SESSION_HOOK.varname,
         TestSessionHooks.SessionHookTest.class.getName());
-    service = new EmbeddedThriftCLIService();
+    service = new EmbeddedThriftBinaryCLIService();
     client = new ThriftCLIServiceClient(service);
   }
 

Added: hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/thrift/TestThriftBinaryCLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/thrift/TestThriftBinaryCLIService.java?rev=1526744&view=auto
==============================================================================
--- hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/thrift/TestThriftBinaryCLIService.java (added)
+++ hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/thrift/TestThriftBinaryCLIService.java Thu Sep 26 23:12:36 2013
@@ -0,0 +1,105 @@
+/**
+ * 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.cli.thrift;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hive.service.auth.HiveAuthFactory.AuthTypes;
+import org.apache.thrift.transport.TTransport;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+
+/**
+ *
+ * TestThriftBinaryCLIService.
+ * This tests ThriftCLIService started in binary mode.
+ *
+ */
+
+public class TestThriftBinaryCLIService extends ThriftCLIServiceTest {
+
+  private static String transportMode = "binary";
+  private static TTransport transport;
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    // Set up the base class
+    ThriftCLIServiceTest.setUpBeforeClass();
+
+    assertNotNull(port);
+    assertNotNull(hiveServer2);
+    assertNotNull(hiveConf);
+
+    hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS, false);
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST, host);
+    hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT, port);
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, AuthTypes.NOSASL.toString());
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, transportMode);
+
+    startHiveServer2WithConf(hiveConf);
+
+    // Open a binary transport
+    // Fail if the transport doesn't open
+    transport = createBinaryTransport();
+    try {
+      transport.open();
+    }
+    catch (Exception e) {
+      fail("Exception: " + e);
+    }
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    ThriftCLIServiceTest.tearDownAfterClass();
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    // Create and set the client
+    initClient(transport);
+    assertNotNull(client);
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @Override
+  @After
+  public void tearDown() throws Exception {
+
+  }
+
+
+}
\ No newline at end of file

Added: hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java?rev=1526744&view=auto
==============================================================================
--- hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java (added)
+++ hive/branches/branch-0.12/service/src/test/org/apache/hive/service/cli/thrift/TestThriftHttpCLIService.java Thu Sep 26 23:12:36 2013
@@ -0,0 +1,216 @@
+/**
+ * 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.cli.thrift;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hive.jdbc.HttpBasicAuthInterceptor;
+import org.apache.hive.service.auth.HiveAuthFactory.AuthTypes;
+import org.apache.hive.service.server.HiveServer2;
+import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.thrift.transport.THttpClient;
+import org.apache.thrift.transport.TTransport;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+*
+* TestThriftHttpCLIService.
+* This tests ThriftCLIService started in http mode.
+*
+*/
+
+public class TestThriftHttpCLIService extends ThriftCLIServiceTest {
+
+  private static String transportMode = "http";
+  private static String thriftHttpPath = "cliservice";
+  private static TTransport transport;
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    // Set up the base class
+    ThriftCLIServiceTest.setUpBeforeClass();
+
+    assertNotNull(port);
+    assertNotNull(hiveServer2);
+    assertNotNull(hiveConf);
+
+    hiveConf.setBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS, false);
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST, host);
+    hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT, port);
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, AuthTypes.NOSASL.toString());
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, transportMode);
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH, thriftHttpPath);
+
+    startHiveServer2WithConf(hiveConf);
+
+    // Open an http transport
+    // Fail if the transport doesn't open
+    transport = createHttpTransport();
+    try {
+      transport.open();
+    }
+    catch (Exception e) {
+      fail("Exception: " + e);
+    }
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    ThriftCLIServiceTest.tearDownAfterClass();
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    // Create and set the client before every test from the transport
+    initClient(transport);
+    assertNotNull(client);
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @Override
+  @After
+  public void tearDown() throws Exception {
+
+  }
+
+  @Test
+  public void testIncompatibeClientServer() throws Exception {
+    // A binary client communicating with an http server should throw an exception
+    // Close the older http client transport
+    // The server is already running in Http mode
+    if (transport != null) {
+      transport.close();
+    }
+    // Create a binary transport and init the client
+    transport = createBinaryTransport();
+    // Create and set the client
+    initClient(transport);
+    assertNotNull(client);
+
+    // This will throw an expected exception since client-server modes are incompatible
+    testOpenSessionExpectedException();
+
+    // Close binary client transport
+    if (transport != null) {
+      transport.close();
+    }
+    // Create http transport (client is inited in setUp before every test from the transport)
+    transport = createHttpTransport();
+    try {
+      transport.open();
+    }
+    catch (Exception e) {
+      fail("Exception: " + e);
+    }
+  }
+
+  @Test
+  public void testIncorrectHttpPath() throws Exception {
+    // Close the older http client transport
+    if (transport != null) {
+      transport.close();
+    }
+    // Create an http transport with incorrect http path endpoint
+    thriftHttpPath = "wrong_path";
+    transport = createHttpTransport();
+    // Create and set the client
+    initClient(transport);
+    assertNotNull(client);
+
+    // This will throw an expected exception since
+    // client is communicating with the wrong http service endpoint
+    testOpenSessionExpectedException();
+
+    // Close incorrect client transport
+    // Reinit http client transport
+    thriftHttpPath = "cliservice";
+    if (transport != null) {
+      transport.close();
+    }
+    transport = createHttpTransport();
+    try {
+      transport.open();
+    }
+    catch (Exception e) {
+      fail("Exception: " + e);
+    }
+  }
+
+
+  private void testWithAuthMode(AuthTypes authType) throws Exception {
+    // Stop and restart HiveServer2 in given incorrect auth mode
+    stopHiveServer2();
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, authType.toString());
+    hiveServer2 = new HiveServer2();
+    // HiveServer2 in Http mode will not start using KERBEROS/LDAP/CUSTOM auth types
+    startHiveServer2WithConf(hiveConf);
+
+    // This will throw an expected exception since Http server is not running
+    testOpenSessionExpectedException();
+
+    // Stop and restart back with the original config
+    stopHiveServer2();
+    hiveConf.setVar(ConfVars.HIVE_SERVER2_AUTHENTICATION, AuthTypes.NOSASL.toString());
+    hiveServer2 = new HiveServer2();
+    startHiveServer2WithConf(hiveConf);
+  }
+
+  @Test
+  public void testKerberosMode()  throws Exception {
+    testWithAuthMode(AuthTypes.KERBEROS);
+  }
+
+  @Test
+  public void testLDAPMode()  throws Exception {
+    testWithAuthMode(AuthTypes.LDAP);
+  }
+
+  @Test
+  public void testCustomMode()  throws Exception {
+    testWithAuthMode(AuthTypes.CUSTOM);
+  }
+
+  private static TTransport createHttpTransport() throws Exception {
+    DefaultHttpClient httpClient = new DefaultHttpClient();
+    String httpUrl = transportMode + "://" + host + ":" + port +
+        "/" + thriftHttpPath + "/";
+    httpClient.addRequestInterceptor(
+        new HttpBasicAuthInterceptor(anonymousUser, anonymousPasswd));
+    return new THttpClient(httpUrl, httpClient);
+  }
+
+}
\ No newline at end of file