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

[hive] branch master updated: HIVE-25479: Browser SSO auth may fail intermittently (Vihang Karajgaonkar, reviewed by Naveen Gangam)"

This is an automated email from the ASF dual-hosted git repository.

vihangk1 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 3e25b52  HIVE-25479: Browser SSO auth may fail intermittently (Vihang Karajgaonkar, reviewed by Naveen Gangam)"
3e25b52 is described below

commit 3e25b524600c62dfde8d069288a162ee6a59d25d
Author: Vihang Karajgaonkar <27...@users.noreply.github.com>
AuthorDate: Tue Oct 19 15:57:58 2021 -0700

    HIVE-25479: Browser SSO auth may fail intermittently (Vihang Karajgaonkar, reviewed by Naveen Gangam)"
    
    Closes  (#2728)
---
 .../jdbc/saml/SimpleSAMLPhpTestBrowserClient.java  |  15 ++
 .../auth/saml/TestHttpSamlAuthentication.java      | 105 ++++++++++++-
 .../java/org/apache/hive/jdbc/HiveConnection.java  |  74 ++++++---
 jdbc/src/java/org/apache/hive/jdbc/Utils.java      |   2 +-
 .../hive/jdbc/saml/HiveJdbcBrowserClient.java      | 169 ++++++++++-----------
 .../hive/jdbc/saml/HttpBrowserClientServlet.java   |  62 ++++++++
 .../jdbc/saml/HttpSamlAuthRequestInterceptor.java  |   5 +
 .../apache/hive/jdbc/saml/IJdbcBrowserClient.java  |  42 +++--
 .../service/auth/saml/HiveSamlHttpServlet.java     |  14 +-
 9 files changed, 345 insertions(+), 143 deletions(-)

diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/saml/SimpleSAMLPhpTestBrowserClient.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/saml/SimpleSAMLPhpTestBrowserClient.java
index 0e2a8b2..97f5de5 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/saml/SimpleSAMLPhpTestBrowserClient.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/saml/SimpleSAMLPhpTestBrowserClient.java
@@ -36,6 +36,7 @@ public class SimpleSAMLPhpTestBrowserClient extends HiveJdbcBrowserClient {
   private final String username;
   private final String password;
   private final long tokenDelayMs;
+  private int injectFailureCount = 0;
   private static final Logger LOG = LoggerFactory
       .getLogger(SimpleSAMLPhpTestBrowserClient.class);
 
@@ -48,6 +49,10 @@ public class SimpleSAMLPhpTestBrowserClient extends HiveJdbcBrowserClient {
     this.tokenDelayMs = tokenDelayMs;
   }
 
+  public void setInjectFailureCount(int injectFailureCount) {
+    this.injectFailureCount = injectFailureCount;
+  }
+
   @Override
   protected void openBrowserWindow() throws HiveJdbcBrowserException {
     // if user and password are null, we fallback to real browser for interactive mode
@@ -73,6 +78,16 @@ public class SimpleSAMLPhpTestBrowserClient extends HiveJdbcBrowserClient {
   }
 
   @Override
+  public void doBrowserSSO() throws HiveJdbcBrowserException {
+    if (injectFailureCount > 0) {
+      injectFailureCount--;
+      throw new HiveJdbcBrowserException(
+          "This is a injected failure for testing purpose");
+    }
+    super.doBrowserSSO();
+  }
+
+  @Override
   public HiveJdbcBrowserServerResponse getServerResponse() {
     if (tokenDelayMs > 0) {
       LOG.debug("Adding a delay of {} msec", tokenDelayMs);
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/service/auth/saml/TestHttpSamlAuthentication.java b/itests/hive-unit/src/test/java/org/apache/hive/service/auth/saml/TestHttpSamlAuthentication.java
index 2a69ed7..d71fded 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/service/auth/saml/TestHttpSamlAuthentication.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/service/auth/saml/TestHttpSamlAuthentication.java
@@ -21,6 +21,7 @@ package org.apache.hive.service.auth.saml;
 import static org.apache.hive.jdbc.Utils.JdbcConnectionParams.AUTH_BROWSER_RESPONSE_PORT;
 import static org.apache.hive.jdbc.Utils.JdbcConnectionParams.AUTH_BROWSER_RESPONSE_TIMEOUT_SECS;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -31,6 +32,10 @@ import com.google.common.io.Files;
 import com.google.common.io.Resources;
 import java.io.File;
 import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
 import java.nio.charset.StandardCharsets;
 import java.sql.ResultSet;
 import java.sql.SQLException;
@@ -48,6 +53,7 @@ import java.util.concurrent.Future;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
+import org.apache.hadoop.hive.ql.metadata.TestHive;
 import org.apache.hive.jdbc.HiveConnection;
 import org.apache.hive.jdbc.Utils.JdbcConnectionParams;
 import org.apache.hive.jdbc.miniHS2.MiniHS2;
@@ -56,6 +62,7 @@ import org.apache.hive.jdbc.saml.IJdbcBrowserClient;
 import org.apache.hive.jdbc.saml.IJdbcBrowserClient.HiveJdbcBrowserException;
 import org.apache.hive.jdbc.saml.IJdbcBrowserClientFactory;
 import org.apache.hive.jdbc.saml.SimpleSAMLPhpTestBrowserClient;
+import org.apache.thrift.TException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -221,6 +228,12 @@ public class TestHttpSamlAuthentication {
         + "=" + responsePort;
   }
 
+  private static String getSamlJdbcConnectionUrlWithRetry(int numRetries)
+      throws Exception {
+    return getSamlJdbcConnectionUrl() + ";" + JdbcConnectionParams.RETRIES + "="
+        + numRetries;
+  }
+
   private static class TestHiveJdbcBrowserClientFactory implements
       IJdbcBrowserClientFactory {
 
@@ -239,8 +252,8 @@ public class TestHttpSamlAuthentication {
     @Override
     public IJdbcBrowserClient create(JdbcConnectionParams connectionParams)
         throws HiveJdbcBrowserException {
-      return new SimpleSAMLPhpTestBrowserClient(connectionParams, user
-          , password, tokenDelayMs);
+      return new SimpleSAMLPhpTestBrowserClient(
+          connectionParams, user, password, tokenDelayMs);
     }
   }
 
@@ -248,7 +261,6 @@ public class TestHttpSamlAuthentication {
    * Test HiveConnection which injects a HTMLUnit based browser client.
    */
   private static class TestHiveConnection extends HiveConnection {
-
     public TestHiveConnection(String uri, Properties info, String testUser,
         String testPass, long tokenDelayMs) throws SQLException {
       super(uri, info, new TestHiveJdbcBrowserClientFactory(testUser, testPass, tokenDelayMs));
@@ -265,6 +277,23 @@ public class TestHttpSamlAuthentication {
     }
   }
 
+  private static class TestHiveConnectionWithInjectedFailure extends TestHiveConnection {
+    private static boolean failureToggle = true;
+    public TestHiveConnectionWithInjectedFailure(String uri, Properties info,
+        String testUser,
+        String testPass) throws SQLException {
+      super(uri, info, testUser, testPass);
+    }
+
+    @Override
+    protected void injectBrowserSSOError() throws Exception {
+      if (failureToggle) {
+        failureToggle = false;
+        throw new Exception("Injected failure");
+      }
+    }
+  }
+
   /**
    * Util class to issue multiple connection requests concurrently.
    */
@@ -376,6 +405,62 @@ public class TestHttpSamlAuthentication {
   }
 
   /**
+   * Makes sure that the port is released once the SSO flow completes. The test retries
+   * twice to make sure that there is no flakiness caused by the race between finding
+   * a free port and immediately some other process acquiring that port later before
+   * Browser client could be started on it.
+   */
+  @Test
+  public void testPortReleaseAfterSSO() throws Exception {
+    setupIDP(true, USER_PASS_MODE);
+    for (int i=0; i<2; i++) {
+      int samlResponsePort = MetaStoreTestUtils.findFreePort();
+      try (TestHiveConnection connection = new TestHiveConnection(
+          getSamlJdbcConnectionUrl(30, samlResponsePort), new Properties(), USER1,
+          USER1_PASSWORD)) {
+        assertLoggedInUser(connection, USER1);
+        assertEquals(samlResponsePort,
+            connection.getBrowserClient().getPort().intValue());
+        assertTrue("Port must be released after SSO flow",
+            isPortAvailable(samlResponsePort));
+        break;
+      }
+    }
+  }
+
+  /**
+   * Here we start HS2 without SAML mode and attempt to use browser SSO against it.
+   * Test makes sure that connection fails and port is released.
+   */
+  @Test
+  public void testPortReleaseOnInvalidConfig() throws Exception {
+    // no IDP setup here; start HS2 in non-SAML mode
+    miniHS2.start(new HashMap<>());
+    int samlResponsePort = MetaStoreTestUtils.findFreePort();
+    try (TestHiveConnection connection = new TestHiveConnection(
+        getSamlJdbcConnectionUrl(30, samlResponsePort), new Properties(), USER1,
+        USER1_PASSWORD)) {
+      fail("Connection should not have succeeded since HS2 is not configured correctly");
+    } catch (Exception e) {
+      assertTrue("Port must be released after SSO flow",
+          isPortAvailable(samlResponsePort));
+    }
+  }
+
+  /**
+   * Make sure that the given port is free by binding to it and then releasing it.
+   * @param port
+   */
+  private boolean isPortAvailable(int port) {
+    try (ServerSocket socket = new ServerSocket(port, 0,
+        InetAddress.getByName(HiveSamlUtils.LOOP_BACK_INTERFACE))) {
+      return true;
+    } catch (IOException e) {
+      return false;
+    }
+  }
+
+  /**
    * Test exercises group name filtering for users. Only users who belong for a given
    * groups should be able to log in. user1 belongs to group1, user2 to group2 and user3
    * to group1 and group2.
@@ -423,6 +508,20 @@ public class TestHttpSamlAuthentication {
     }
   }
 
+
+  /**
+   * Test injects failure in the first connection attempt and then makes sure that the
+   * 2nd retry works as expected.
+   */
+  @Test
+  public void testConnectionRetries() throws Exception {
+    setupIDP(true, USER_PASS_MODE);
+    try (HiveConnection connection = new TestHiveConnectionWithInjectedFailure(
+        getSamlJdbcConnectionUrlWithRetry(2), new Properties(), USER1, USER1_PASSWORD)) {
+      assertLoggedInUser(connection, USER1);
+    }
+  }
+
   /**
    * Test makes sure that the token received in the server response is not valid
    * after it is expired.
diff --git a/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java b/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
index 4eefded..5fcad06 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
@@ -179,7 +179,7 @@ public class HiveConnection implements java.sql.Connection {
   private Properties clientInfo;
   private Subject loggedInSubject;
   private int maxRetries = 1;
-  private final IJdbcBrowserClient browserClient;
+  private IJdbcBrowserClient browserClient;
 
   /**
    * Get all direct HiveServer2 URLs from a ZooKeeper based HiveServer2 URL
@@ -276,7 +276,6 @@ public class HiveConnection implements java.sql.Connection {
     sessConfMap = null;
     isEmbeddedMode = true;
     initFetchSize = 0;
-    browserClient = null;
   }
 
   public HiveConnection(String uri, Properties info) throws SQLException {
@@ -284,6 +283,11 @@ public class HiveConnection implements java.sql.Connection {
   }
 
   @VisibleForTesting
+  protected int getNumRetries() {
+    return maxRetries;
+  }
+
+  @VisibleForTesting
   protected HiveConnection(String uri, Properties info,
       IJdbcBrowserClientFactory browserClientFactory) throws SQLException {
     try {
@@ -340,8 +344,6 @@ public class HiveConnection implements java.sql.Connection {
       } catch (HiveJdbcBrowserException e) {
         throw new SQLException("");
       }
-    } else {
-      browserClient = null;
     }
     if (isEmbeddedMode) {
       client = EmbeddedCLIServicePortal.get(connParams.getHiveConfs());
@@ -1067,33 +1069,58 @@ public class HiveConnection implements java.sql.Connection {
     //TODO This is a bit hacky. We piggy back on a dummy OpenSession call
     // to get the redirect response from the server. Instead its probably cleaner to
     // explicitly do a HTTP post request and get the response.
-    int numRetry = isBrowserAuthMode() ? 2 : 1;
-    for (int i=0; i<numRetry; i++) {
-      try {
-        openSession(openReq);
-      } catch (TException e) {
-        if (isSamlRedirect(e)) {
-          boolean success = doBrowserSSO();
-          if (!success) {
-            String msg = browserClient.getServerResponse() == null
-                || browserClient.getServerResponse().getMsg() == null ? ""
-                : browserClient.getServerResponse().getMsg();
+    try {
+      int numRetry = 1;
+      if (isBrowserAuthMode()) {
+        numRetry = 2;
+        browserClient.startListening();
+      }
+      for (int i=0; i<numRetry; i++) {
+        try {
+          openSession(openReq);
+        } catch (TException e) {
+          if (isSamlRedirect(e)) {
+            boolean success = doBrowserSSO();
+            if (!success) {
+              String msg = browserClient.getServerResponse() == null
+                  || browserClient.getServerResponse().getMsg() == null ? ""
+                  : browserClient.getServerResponse().getMsg();
+              throw new SQLException(
+                  "Could not establish connection to " + jdbcUriString + ": "
+                      + msg, " 08S01", e);
+            }
+          } else {
             throw new SQLException(
-                "Could not establish connection to " + jdbcUriString + ": "
-                    + msg, " 08S01", e);
+                "Could not establish connection to " + jdbcUriString + ": " + e
+                    .getMessage(), " 08S01", e);
           }
-        } else {
-          throw new SQLException(
-              "Could not establish connection to " + jdbcUriString + ": " + e
-                  .getMessage(), " 08S01", e);
+        }
+      }
+    } catch (HiveJdbcBrowserException e) {
+      throw new SQLException(
+          "Could not establish connection to " + jdbcUriString + ": " + e
+              .getMessage(), " 08S01", e);
+    } finally {
+      if (browserClient != null) {
+        try {
+          browserClient.close();
+        } catch (IOException e) {
+          LOG.error("Unable to close the browser SSO client : " + e.getMessage(), e);
         }
       }
     }
     isClosed = false;
   }
 
-  private boolean doBrowserSSO() throws SQLException {
+  @VisibleForTesting
+  protected void injectBrowserSSOError() throws Exception {
+    //no-op
+  }
+
+  @VisibleForTesting
+  protected boolean doBrowserSSO() throws SQLException {
     try {
+      injectBrowserSSOError();
       Preconditions.checkNotNull(browserClient);
       try (IJdbcBrowserClient bc = browserClient) {
         browserClient.doBrowserSSO();
@@ -1105,8 +1132,7 @@ public class HiveConnection implements java.sql.Connection {
       }
     } catch (Exception ex) {
       throw new SQLException("Browser based SSO failed: " + ex.getMessage(),
-          " 08S01",
-          ex);
+          " 08S01", ex);
     }
   }
 
diff --git a/jdbc/src/java/org/apache/hive/jdbc/Utils.java b/jdbc/src/java/org/apache/hive/jdbc/Utils.java
index ffd628c..0fd4820 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/Utils.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/Utils.java
@@ -85,7 +85,7 @@ public class Utils {
     // Client param names:
 
     // Retry setting
-    static final String RETRIES = "retries";
+    public static final String RETRIES = "retries";
     public static final String RETRY_INTERVAL = "retryInterval";
 
     public static final String AUTH_TYPE = "auth";
diff --git a/jdbc/src/java/org/apache/hive/jdbc/saml/HiveJdbcBrowserClient.java b/jdbc/src/java/org/apache/hive/jdbc/saml/HiveJdbcBrowserClient.java
index 28732fd..c03c961 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/saml/HiveJdbcBrowserClient.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/saml/HiveJdbcBrowserClient.java
@@ -20,19 +20,12 @@ package org.apache.hive.jdbc.saml;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import java.awt.Desktop;
 import java.awt.Desktop.Action;
-import java.io.BufferedReader;
 import java.io.IOException;
-import java.io.InputStreamReader;
 import java.io.PrintWriter;
 import java.io.UnsupportedEncodingException;
-import java.net.InetAddress;
-import java.net.ServerSocket;
 import java.net.Socket;
-import java.net.SocketTimeoutException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URLDecoder;
@@ -41,12 +34,15 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
 import org.apache.hive.jdbc.Utils.JdbcConnectionParams;
 import org.apache.hive.service.auth.saml.HiveSamlUtils;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.ServletHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -60,48 +56,80 @@ public class HiveJdbcBrowserClient implements IJdbcBrowserClient {
   // error message when the socket times out.
   @VisibleForTesting
   public static final String TIMEOUT_ERROR_MSG = "Timed out while waiting for server response";
-  private final ServerSocket serverSocket;
-  private HiveJdbcBrowserServerResponse serverResponse;
+  // port as parsed from the connection URL; default is 0 which means any available port.
+  private final int portFromUrl;
+  // the actual port on the local machine where the web server is running
+  private Integer serverPort;
+  // timeout in mill-sec until which browserClient will wait for auth response from the
+  // HS2 server.
+  private final long timeoutInMs;
+  private final BlockingQueue<HiveJdbcBrowserServerResponse>
+      serverResponseQueue = new LinkedBlockingDeque<>();
   protected JdbcBrowserClientContext clientContext;
   // By default we wait for 2 min unless overridden by a JDBC connection param
   // browserResponseTimeout
   private static final int DEFAULT_SOCKET_TIMEOUT_SECS = 120;
-  private final ExecutorService serverResponseThread = Executors.newSingleThreadExecutor(
-      new ThreadFactoryBuilder().setNameFormat("Hive-Jdbc-Browser-Client-%d")
-          .setDaemon(true).build());
+  private Server webServer;
+  private HiveJdbcBrowserServerResponse serverResponse;
 
   HiveJdbcBrowserClient(JdbcConnectionParams connectionParams)
       throws HiveJdbcBrowserException {
-    serverSocket = getServerSocket(connectionParams.getSessionVars());
+    portFromUrl = Integer.parseInt(connectionParams.getSessionVars()
+        .getOrDefault(JdbcConnectionParams.AUTH_BROWSER_RESPONSE_PORT, "0"));
+    timeoutInMs = Integer.parseInt(
+        connectionParams.getSessionVars()
+            .getOrDefault(JdbcConnectionParams.AUTH_BROWSER_RESPONSE_TIMEOUT_SECS,
+                String.valueOf(DEFAULT_SOCKET_TIMEOUT_SECS))) * 1000L;
   }
 
-  private ServerSocket getServerSocket(Map<String, String> sessionConf)
-      throws HiveJdbcBrowserException {
-    final ServerSocket serverSocket;
-    int port = Integer.parseInt(sessionConf
-        .getOrDefault(JdbcConnectionParams.AUTH_BROWSER_RESPONSE_PORT, "0"));
-    int timeout = Integer.parseInt(
-        sessionConf.getOrDefault(JdbcConnectionParams.AUTH_BROWSER_RESPONSE_TIMEOUT_SECS,
-            String.valueOf(DEFAULT_SOCKET_TIMEOUT_SECS)));
+  @Override
+  public void startListening() throws HiveJdbcBrowserException {
+    webServer = new Server();
+    ServerConnector serverConnector = new ServerConnector(webServer);
+    serverConnector.setHost(HiveSamlUtils.LOOP_BACK_INTERFACE);
+    serverConnector.setPort(portFromUrl);
+    LOG.info("Browser response timeout is set to {} ms", timeoutInMs);
+    serverConnector.setIdleTimeout(timeoutInMs);
+    webServer.addConnector(serverConnector);
+    ServletHandler servletHandler = new ServletHandler();
+    servletHandler.addServletWithMapping(
+        new ServletHolder(new HttpBrowserClientServlet(this)), "/");
+    webServer.setHandler(servletHandler);
+    webServer.setStopTimeout(30*1000L);
     try {
-      serverSocket = new ServerSocket(port, 0,
-          InetAddress.getByName(HiveSamlUtils.LOOP_BACK_INTERFACE));
-      LOG.debug("Browser response timeout is set to {} seconds", timeout);
-      serverSocket.setSoTimeout(timeout * 1000);
-    } catch (IOException e) {
-      throw new HiveJdbcBrowserException("Unable to bind to the localhost");
+      webServer.start();
+      // we fetch the port after the server is started so that we can get the port
+      // where the server has bound in case there is no port specified from the URL.
+      serverPort = ((ServerConnector) webServer.getConnectors()[0]).getLocalPort();
+      LOG.debug("Listening on the port {} ", serverPort);
+    } catch (Exception e) {
+      throw new HiveJdbcBrowserException("Could not start http server", e);
     }
-    return serverSocket;
   }
 
   public Integer getPort() {
-    return serverSocket.getLocalPort();
+    return serverPort;
+  }
+
+  @Override
+  public String toString() {
+    return "HiveJdbcBrowserClient@" + serverPort;
+  }
+
+  @Override
+  public HiveJdbcBrowserServerResponse getServerResponse() {
+    return serverResponse;
   }
 
   @Override
   public void close() throws IOException {
-    if (serverSocket != null) {
-      serverSocket.close();
+    if (webServer != null && webServer.isRunning()) {
+      try {
+        webServer.stop();
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+      webServer = null;
     }
   }
 
@@ -111,7 +139,7 @@ public class HiveJdbcBrowserClient implements IJdbcBrowserClient {
     // expired
     reset();
     this.clientContext = clientContext;
-    LOG.trace("Initialized the JDBCBrowser client with URL {}",
+    LOG.debug("Initialized the JDBCBrowser client with URL {}",
         clientContext.getSsoUri());
   }
 
@@ -121,14 +149,20 @@ public class HiveJdbcBrowserClient implements IJdbcBrowserClient {
   }
 
   public void doBrowserSSO() throws HiveJdbcBrowserException {
-    Future<Void> serverResponseHandle = waitAsyncForServerResponse();
     logDebugInfoUri(clientContext.getSsoUri());
     openBrowserWindow();
     try {
-      serverResponseHandle.get();
-    } catch (InterruptedException | ExecutionException e) {
+      waitForServerResponse(timeoutInMs);
+    } catch (InterruptedException e) {
       throw new HiveJdbcBrowserException(e);
     }
+    if (serverResponse == null) {
+      throw new HiveJdbcBrowserException(TIMEOUT_ERROR_MSG);
+    }
+    if (!serverResponse.isValid()) {
+      throw new HiveJdbcBrowserException(
+          "Received invalid response from server. See driver logs for more details");
+    }
   }
 
   private void logDebugInfoUri(URI ssoURI) {
@@ -136,7 +170,7 @@ public class HiveJdbcBrowserClient implements IJdbcBrowserClient {
     try {
       uriParams = getQueryParams(ssoURI);
     } catch (HiveJdbcBrowserException e) {
-      LOG.debug("Could get query params of the SSO URI", e);
+      LOG.info("Could get query params of the SSO URI", e);
     }
     LOG.debug("Initializing browser SSO request to URI. Relay state is {}",
         uriParams.get("RelayState"));
@@ -175,7 +209,7 @@ public class HiveJdbcBrowserClient implements IJdbcBrowserClient {
           .isSupported(Action.BROWSE)) {
         Desktop.getDesktop().browse(ssoUri);
       } else {
-        LOG.debug(
+        LOG.info(
             "Desktop mode is not supported. Attempting to use OS "
                 + "commands to open the default browser");
         //Desktop is not supported, lets try to open the browser process
@@ -201,52 +235,17 @@ public class HiveJdbcBrowserClient implements IJdbcBrowserClient {
     }
   }
 
-  private Future<Void> waitAsyncForServerResponse() {
-    return serverResponseThread.submit(() -> {
-      // listen to the response on the server socket
-      Socket socket;
-      try {
-        LOG.debug("Waiting for a server response on port {} with a timeout of {} ms",
-            serverSocket.getLocalPort(), serverSocket.getSoTimeout());
-        socket = serverSocket.accept();
-      } catch (SocketTimeoutException timeoutException) {
-        throw new HiveJdbcBrowserException(TIMEOUT_ERROR_MSG,
-            timeoutException);
-      } catch (IOException e) {
-        throw new HiveJdbcBrowserException(
-            "Unexpected error while listening on port " + serverSocket.getLocalPort()
-                + " for server response", e);
-      }
-      try (BufferedReader reader = new BufferedReader(new InputStreamReader(
-          socket.getInputStream(), StandardCharsets.UTF_8))) {
-        char[] buffer = new char[16 * 1024];
-        // block until you read into the buffer
-        int len = reader.read(buffer);
-        String response = String.valueOf(buffer, 0, len);
-        String[] lines = response.split("\r\n");
-        for (String line : lines) {
-          if (!Strings.isNullOrEmpty(line)) {
-            if (line.contains("token=")) {
-              serverResponse = new HiveJdbcBrowserServerResponse(line);
-              sendBrowserMsg(socket, serverResponse.isSuccessful());
-            } else {
-              LOG.trace("Skipping line {} from server response", line);
-            }
-          }
-        }
-        if (serverResponse == null) {
-          throw new HiveJdbcBrowserException("Could not parse the response from server.");
-        }
-      } catch (IOException e) {
-        throw new HiveJdbcBrowserException(
-            "Unexpected exception while processing server response ", e);
-      }
-      return null;
-    });
+  public void addServerResponse(HiveJdbcBrowserServerResponse response) {
+    serverResponseQueue.add(response);
   }
 
-  public HiveJdbcBrowserServerResponse getServerResponse() {
-    return serverResponse;
+  /**
+   * Waits for a server response until the given timeout value in milliseconds.
+   * Returns null if timeout.
+   */
+  private void waitForServerResponse(long timeoutInMs)
+      throws InterruptedException {
+     serverResponse = serverResponseQueue.poll(timeoutInMs, TimeUnit.MILLISECONDS);
   }
 
   public String getClientIdentifier() {
diff --git a/jdbc/src/java/org/apache/hive/jdbc/saml/HttpBrowserClientServlet.java b/jdbc/src/java/org/apache/hive/jdbc/saml/HttpBrowserClientServlet.java
new file mode 100644
index 0000000..f83fd95
--- /dev/null
+++ b/jdbc/src/java/org/apache/hive/jdbc/saml/HttpBrowserClientServlet.java
@@ -0,0 +1,62 @@
+/*
+ * 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.saml;
+
+import java.io.IOException;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.hive.jdbc.saml.IJdbcBrowserClient.HiveJdbcBrowserServerResponse;
+import org.apache.hive.service.auth.saml.HiveSamlUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HttpBrowserClientServlet extends HttpServlet {
+  private final HiveJdbcBrowserClient browserClient;
+  private static final Logger LOG = LoggerFactory.getLogger(
+      HttpBrowserClientServlet.class);
+  HttpBrowserClientServlet(HiveJdbcBrowserClient browserClient) {
+    this.browserClient = browserClient;
+  }
+
+  @Override
+  protected void doPost(
+      HttpServletRequest req, HttpServletResponse resp)
+      throws ServletException, IOException {
+    LOG.info("Received request from {}:{} for browserClient at port {}",
+        req.getRemoteAddr(), req.getRemotePort(), browserClient.toString());
+    resp.setContentType("text/html");
+    resp.setStatus(HttpServletResponse.SC_OK);
+    String token = req.getParameter(HiveSamlUtils.TOKEN_KEY);
+    String msg = req.getParameter(HiveSamlUtils.MESSAGE_KEY);
+    boolean status = Boolean.parseBoolean(req.getParameter(HiveSamlUtils.STATUS_KEY));
+    HiveJdbcBrowserServerResponse response = new HiveJdbcBrowserServerResponse(status,
+        msg, token);
+    if (response.isSuccessful()) {
+      resp.getWriter().write("Successfully authenticated. You may close this window.");
+    } else {
+      resp.getWriter().write(
+          "Authentication failed. Please check server logs for details. "
+              + "You may close this window.");
+    }
+    resp.getWriter().flush();
+    browserClient.addServerResponse(response);
+  }
+}
diff --git a/jdbc/src/java/org/apache/hive/jdbc/saml/HttpSamlAuthRequestInterceptor.java b/jdbc/src/java/org/apache/hive/jdbc/saml/HttpSamlAuthRequestInterceptor.java
index 9ada4e5..ad3d89d 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/saml/HttpSamlAuthRequestInterceptor.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/saml/HttpSamlAuthRequestInterceptor.java
@@ -26,6 +26,8 @@ import org.apache.http.HttpHeaders;
 import org.apache.http.HttpRequest;
 import org.apache.http.client.CookieStore;
 import org.apache.http.protocol.HttpContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This implements the logic to intercept the HTTP requests from the Hive Jdbc connection
@@ -36,6 +38,8 @@ public class HttpSamlAuthRequestInterceptor extends HttpRequestInterceptorBase {
 
   private final IJdbcBrowserClient browserClient;
   private static final String BEARER = "Bearer ";
+  private static final Logger LOG = LoggerFactory
+      .getLogger(HttpSamlAuthRequestInterceptor.class);
 
   public HttpSamlAuthRequestInterceptor(IJdbcBrowserClient browserClient, CookieStore cookieStore, String cn,
       boolean isSSL, Map<String, String> additionalHeaders,
@@ -56,6 +60,7 @@ public class HttpSamlAuthRequestInterceptor extends HttpRequestInterceptorBase {
       httpRequest.addHeader(HiveSamlUtils.SSO_CLIENT_IDENTIFIER, clientIdentifier);
       httpRequest.removeHeaders(HiveSamlUtils.SSO_TOKEN_RESPONSE_PORT);
     } else {
+      LOG.debug("Adding response port {}", port);
       httpRequest.addHeader(HiveSamlUtils.SSO_TOKEN_RESPONSE_PORT, port);
     }
   }
diff --git a/jdbc/src/java/org/apache/hive/jdbc/saml/IJdbcBrowserClient.java b/jdbc/src/java/org/apache/hive/jdbc/saml/IJdbcBrowserClient.java
index c35aeca..6bed420 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/saml/IJdbcBrowserClient.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/saml/IJdbcBrowserClient.java
@@ -36,6 +36,12 @@ import org.apache.hive.service.auth.saml.HiveSamlUtils;
 public interface IJdbcBrowserClient extends Closeable {
 
   /**
+   * Start a webserver and bind to a port number as configued.
+   * @throws HiveJdbcBrowserException
+   */
+  void startListening() throws HiveJdbcBrowserException;
+
+  /**
    * Execute the browser actions to complete the SSO workflow. This method assumes
    * that the {@link #init(JdbcBrowserClientContext)} method has been called already
    * to initialize the state needed for doing the browser based flow.
@@ -111,30 +117,10 @@ public interface IJdbcBrowserClient extends Closeable {
     private final boolean status;
     private final String token;
 
-    public HiveJdbcBrowserServerResponse(String postResponse) {
-      Map<String, String> params = parseUrlEncodedFormData(postResponse);
-      status = Boolean.parseBoolean(params.get(HiveSamlUtils.STATUS_KEY));
-      msg = params.getOrDefault(HiveSamlUtils.MESSAGE_KEY, "");
-      token = params.get(HiveSamlUtils.TOKEN_KEY);
-    }
-
-
-    private Map<String, String> parseUrlEncodedFormData(String line) {
-      String decoded;
-      try {
-        decoded = URLDecoder.decode(line, StandardCharsets.UTF_8.toString());
-      } catch (UnsupportedEncodingException e) {
-        throw new RuntimeException(e);
-      }
-      Map<String, String> ret = new HashMap<>();
-      for (String params : decoded.split("&")) {
-        if (params.contains("=")) {
-          String key = params.substring(0, params.indexOf("="));
-          String val = params.substring(params.indexOf("=") + 1);
-          ret.put(key, val);
-        }
-      }
-      return ret;
+    public HiveJdbcBrowserServerResponse(boolean status, String msg, String token) {
+      this.status = status;
+      this.msg = msg;
+      this.token = token;
     }
 
     public String getMsg() {
@@ -148,5 +134,13 @@ public interface IJdbcBrowserClient extends Closeable {
     public String getToken() {
       return token;
     }
+
+    /**
+     * A response is valid if the status is true with a non-empty token or
+     * if status is false
+     */
+    public boolean isValid() {
+      return !status || (token != null && !token.isEmpty());
+    }
   }
 }
diff --git a/service/src/java/org/apache/hive/service/auth/saml/HiveSamlHttpServlet.java b/service/src/java/org/apache/hive/service/auth/saml/HiveSamlHttpServlet.java
index a206e91..4527ed5 100644
--- a/service/src/java/org/apache/hive/service/auth/saml/HiveSamlHttpServlet.java
+++ b/service/src/java/org/apache/hive/service/auth/saml/HiveSamlHttpServlet.java
@@ -35,7 +35,6 @@ public class HiveSamlHttpServlet extends HttpServlet {
       .getLogger(HiveSamlHttpServlet.class);
   private final HiveConf conf;
   private final ISAMLAuthTokenGenerator tokenGenerator;
-  private static final String LOOP_BACK_INTERFACE = "127.0.0.1";
 
   public HiveSamlHttpServlet(HiveConf conf) {
     this.conf = Preconditions.checkNotNull(conf);
@@ -56,7 +55,8 @@ public class HiveSamlHttpServlet extends HttpServlet {
       return;
     }
     try {
-      LOG.info("RelayState = {}. Driver side port on localhost = {}", relayState, port);
+      LOG.info("RelayState = {}. Driver side port on loopback address is {}", relayState,
+          port);
       nameId = HiveSaml2Client.get(conf).validate(request, response);
     } catch (HttpSamlAuthenticationException e) {
       if (e instanceof HttpSamlNoGroupsMatchedException) {
@@ -77,7 +77,7 @@ public class HiveSamlHttpServlet extends HttpServlet {
   }
 
   private void generateFormData(HttpServletResponse response, String url, String token,
-      boolean sucess, String msg) {
+      boolean success, String msg) {
     StringBuilder sb = new StringBuilder();
     sb.append("<html>");
     sb.append("<body onload='document.forms[\"form\"].submit()'>");
@@ -86,15 +86,17 @@ public class HiveSamlHttpServlet extends HttpServlet {
         .format("<input type='hidden' name='%s' value='%s'>", HiveSamlUtils.TOKEN_KEY,
             token));
     sb.append(String.format("<input type='hidden' name='%s' value='%s'>",
-        HiveSamlUtils.STATUS_KEY, sucess));
+        HiveSamlUtils.STATUS_KEY, success));
     sb.append(String
         .format("<input type='hidden' name='%s' value='%s'>", HiveSamlUtils.MESSAGE_KEY,
             msg));
     sb.append("</form>");
     sb.append("</body>");
     sb.append("</html>");
-    try (PrintWriter write = response.getWriter()) {
-      write.write(sb.toString());
+    response.setContentType("text/html");
+    try {
+      response.getWriter().write(sb.toString());
+      response.getWriter().flush();
     } catch (IOException e) {
       LOG.error("Could not generate the form data for sending a response to url " + url,
           e);