You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by vg...@apache.org on 2018/04/02 18:36:24 UTC

hive git commit: HIVE-18963: JDBC: Provide an option to simplify beeline usage by supporting default and named URL for beeline (Vaibhav Gumashta reviewed by Vihang Karajgaonkar, Thejas Nair)

Repository: hive
Updated Branches:
  refs/heads/master 733aecf28 -> 54dbd7fe2


HIVE-18963: JDBC: Provide an option to simplify beeline usage by supporting default and named URL for beeline (Vaibhav Gumashta reviewed by Vihang Karajgaonkar, Thejas Nair)


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

Branch: refs/heads/master
Commit: 54dbd7fe2fe37fc9f9b0721fde4dd327e4011bc9
Parents: 733aecf
Author: Vaibhav Gumashta <vg...@hortonworks.com>
Authored: Mon Apr 2 11:28:30 2018 -0700
Committer: Vaibhav Gumashta <vg...@hortonworks.com>
Committed: Mon Apr 2 11:28:30 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hive/beeline/BeeLine.java   | 108 +++++++---
 .../BeelineConfFileParseException.java          |  30 +++
 .../BeelineHS2ConnectionFileParseException.java |   2 +-
 .../BeelineSiteParseException.java              |  30 +++
 .../hs2connection/BeelineSiteParser.java        | 145 +++++++++++++
 .../hs2connection/HS2ConnectionFileParser.java  |   2 +-
 .../hs2connection/HS2ConnectionFileUtils.java   | 131 ++++++++++++
 .../UserHS2ConnectionFileParser.java            |   3 -
 beeline/src/main/resources/BeeLine.properties   |   3 +
 .../TestUserHS2ConnectionFileParser.java        |   4 +-
 .../BeelineWithHS2ConnectionFileTestBase.java   |   2 +-
 jdbc/src/java/org/apache/hive/jdbc/Utils.java   | 212 ++++++++++---------
 12 files changed, 536 insertions(+), 136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/54dbd7fe/beeline/src/java/org/apache/hive/beeline/BeeLine.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
index 4928761..6f7f1fc 100644
--- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java
+++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
@@ -35,7 +35,6 @@ import java.io.PrintStream;
 import java.io.SequenceInputStream;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
 import java.net.JarURLConnection;
 import java.net.URL;
 import java.net.URLConnection;
@@ -75,9 +74,6 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.jar.Attributes;
 import java.util.jar.Manifest;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipFile;
-
 import jline.console.completer.Completer;
 import jline.console.completer.StringsCompleter;
 import jline.console.completer.FileNameCompleter;
@@ -95,7 +91,9 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hive.beeline.cli.CliOptionsProcessor;
 import org.apache.hive.common.util.ShutdownHookManager;
-import org.apache.hive.beeline.hs2connection.BeelineHS2ConnectionFileParseException;
+import org.apache.hive.beeline.hs2connection.BeelineConfFileParseException;
+import org.apache.hive.beeline.hs2connection.BeelineSiteParseException;
+import org.apache.hive.beeline.hs2connection.BeelineSiteParser;
 import org.apache.hive.beeline.hs2connection.HS2ConnectionFileUtils;
 import org.apache.hive.beeline.hs2connection.UserHS2ConnectionFileParser;
 import org.apache.hive.beeline.hs2connection.HS2ConnectionFileParser;
@@ -104,6 +102,7 @@ import org.apache.thrift.transport.TTransportException;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import org.apache.hive.jdbc.JdbcUriParseException;
 import org.apache.hive.jdbc.Utils;
 import org.apache.hive.jdbc.Utils.JdbcConnectionParams;
 
@@ -309,16 +308,24 @@ public class BeeLine implements Closeable {
     options.addOption(OptionBuilder
         .hasArg()
         .withArgName("driver class")
-        .withDescription("the driver class to use")
+        .withDescription("The driver class to use")
         .create('d'));
 
     // -u <database url>
     options.addOption(OptionBuilder
         .hasArg()
         .withArgName("database url")
-        .withDescription("the JDBC URL to connect to")
+        .withDescription("The JDBC URL to connect to")
         .create('u'));
 
+    // -c <named url in the beeline-hs2-connection.xml>
+    options.addOption(OptionBuilder
+        .hasArg()
+        .withArgName("named JDBC URL in beeline-site.xml")
+        .withDescription("The named JDBC URL to connect to, which should be present in "
+            + "beeline-site.xml as the value of beeline.hs2.jdbc.url.<namedUrl>")
+        .create('c'));
+
     // -r
     options.addOption(OptionBuilder
         .withLongOpt("reconnect")
@@ -329,14 +336,14 @@ public class BeeLine implements Closeable {
     options.addOption(OptionBuilder
         .hasArg()
         .withArgName("username")
-        .withDescription("the username to connect as")
+        .withDescription("The username to connect as")
         .create('n'));
 
     // -p <password>
     options.addOption(OptionBuilder
         .hasArg()
         .withArgName("password")
-        .withDescription("the password to connect as")
+        .withDescription("The password to connect as")
         .hasOptionalArg()
         .create('p'));
 
@@ -344,7 +351,7 @@ public class BeeLine implements Closeable {
     options.addOption(OptionBuilder
         .hasArg()
         .withArgName("password-file")
-        .withDescription("the password file to read password from")
+        .withDescription("The password file to read password from")
         .withLongOpt("password-file")
         .create('w'));
 
@@ -352,34 +359,34 @@ public class BeeLine implements Closeable {
     options.addOption(OptionBuilder
         .hasArg()
         .withArgName("authType")
-        .withDescription("the authentication type")
+        .withDescription("The authentication type")
         .create('a'));
 
     // -i <init file>
     options.addOption(OptionBuilder
         .hasArgs()
         .withArgName("init")
-        .withDescription("script file for initialization")
+        .withDescription("The script file for initialization")
         .create('i'));
 
     // -e <query>
     options.addOption(OptionBuilder
         .hasArgs()
         .withArgName("query")
-        .withDescription("query that should be executed")
+        .withDescription("The query that should be executed")
         .create('e'));
 
     // -f <script file>
     options.addOption(OptionBuilder
         .hasArg()
         .withArgName("file")
-        .withDescription("script file that should be executed")
+        .withDescription("The script file that should be executed")
         .create('f'));
 
     // -help
     options.addOption(OptionBuilder
         .withLongOpt("help")
-        .withDescription("display this message")
+        .withDescription("Display this message")
         .create('h'));
 
     // Substitution option --hivevar
@@ -388,7 +395,7 @@ public class BeeLine implements Closeable {
         .hasArgs(2)
         .withArgName("key=value")
         .withLongOpt("hivevar")
-        .withDescription("hive variable name and value")
+        .withDescription("Hive variable name and value")
         .create());
 
     //hive conf option --hiveconf
@@ -404,7 +411,7 @@ public class BeeLine implements Closeable {
     options.addOption(OptionBuilder
         .hasArg()
         .withLongOpt("property-file")
-        .withDescription("the file to read configuration properties from")
+        .withDescription("The file to read configuration properties from")
         .create());
   }
 
@@ -789,7 +796,7 @@ public class BeeLine implements Closeable {
     // and uses it to connect if found
     // no-op if the file is not present
     if(!connSuccessful && !exit) {
-      connSuccessful = defaultBeelineConnect();
+      connSuccessful = defaultBeelineConnect(cl);
     }
 
     int code = 0;
@@ -1064,50 +1071,93 @@ public class BeeLine implements Closeable {
    * if there connection is not made return false
    *
    */
-  private boolean defaultBeelineConnect() {
+  private boolean defaultBeelineConnect(CommandLine cl) {
     String url;
     try {
-      url = getDefaultConnectionUrl();
+      url = getDefaultConnectionUrl(cl);
       if (url == null) {
         debug("Default hs2 connection config file not found");
         return false;
       }
-    } catch (BeelineHS2ConnectionFileParseException e) {
+    } catch (BeelineConfFileParseException e) {
       error(e);
       return false;
     }
     return dispatch("!connect " + url);
   }
 
+  private String getDefaultConnectionUrl(CommandLine cl) throws BeelineConfFileParseException {
+    Properties mergedConnectionProperties = new Properties();
+    JdbcConnectionParams jdbcConnectionParams = null;
+    BeelineSiteParser beelineSiteParser = getUserBeelineSiteParser();
+    UserHS2ConnectionFileParser userHS2ConnFileParser = getUserHS2ConnFileParser();
+    Properties userConnectionProperties = new Properties();
 
-  private String getDefaultConnectionUrl() throws BeelineHS2ConnectionFileParseException {
-    HS2ConnectionFileParser userHS2ConnFileParser = getUserHS2ConnFileParser();
-    if (!userHS2ConnFileParser.configExists()) {
+    if (!userHS2ConnFileParser.configExists() && !beelineSiteParser.configExists()) {
       // nothing to do if there is no user HS2 connection configuration file
+      // or beeline-site.xml in the path
       return null;
     }
-    // get the connection properties from user specific config file
-    Properties userConnectionProperties = userHS2ConnFileParser.getConnectionProperties();
+
+    if (beelineSiteParser.configExists()) {
+      // Get the named url from user specific config file if present
+      Properties userNamedConnectionURLs = beelineSiteParser.getConnectionProperties();
+      if (!userNamedConnectionURLs.isEmpty()) {
+        String urlName = cl.getOptionValue("c");
+        String jdbcURL = HS2ConnectionFileUtils.getNamedUrl(userNamedConnectionURLs, urlName);
+        if (jdbcURL != null) {
+          try {
+            jdbcConnectionParams = Utils.extractURLComponents(jdbcURL, new Properties());
+          } catch (JdbcUriParseException e) {
+            throw new BeelineSiteParseException(
+                "Error in parsing jdbc url: " + jdbcURL + " from beeline-site.xml", e);
+          }
+        }
+      }
+    }
+
+    if (userHS2ConnFileParser.configExists()) {
+      // get the connection properties from user specific config file
+      userConnectionProperties = userHS2ConnFileParser.getConnectionProperties();
+    }
+
+    if (jdbcConnectionParams != null) {
+      mergedConnectionProperties =
+          HS2ConnectionFileUtils.mergeUserConnectionPropertiesAndBeelineSite(
+              userConnectionProperties, jdbcConnectionParams);
+    } else {
+      mergedConnectionProperties = userConnectionProperties;
+    }
+
     // load the HS2 connection url properties from hive-site.xml if it is present in the classpath
     HS2ConnectionFileParser hiveSiteParser = getHiveSiteHS2ConnectionFileParser();
     Properties hiveSiteConnectionProperties = hiveSiteParser.getConnectionProperties();
     // add/override properties found from hive-site with user-specific properties
-    for (String key : userConnectionProperties.stringPropertyNames()) {
+    for (String key : mergedConnectionProperties.stringPropertyNames()) {
       if (hiveSiteConnectionProperties.containsKey(key)) {
         debug("Overriding connection url property " + key
             + " from user connection configuration file");
       }
-      hiveSiteConnectionProperties.setProperty(key, userConnectionProperties.getProperty(key));
+      hiveSiteConnectionProperties.setProperty(key, mergedConnectionProperties.getProperty(key));
     }
     // return the url based on the aggregated connection properties
     return HS2ConnectionFileUtils.getUrl(hiveSiteConnectionProperties);
   }
 
+
+  /*
+   * Increased visibility of this method is only for providing better test coverage
+   */
+  @VisibleForTesting
+  public BeelineSiteParser getUserBeelineSiteParser() {
+    return new BeelineSiteParser();
+  }
+
   /*
    * Increased visibility of this method is only for providing better test coverage
    */
   @VisibleForTesting
-  public HS2ConnectionFileParser getUserHS2ConnFileParser() {
+  public UserHS2ConnectionFileParser getUserHS2ConnFileParser() {
     return new UserHS2ConnectionFileParser();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/54dbd7fe/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineConfFileParseException.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineConfFileParseException.java b/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineConfFileParseException.java
new file mode 100644
index 0000000..4b42ee4
--- /dev/null
+++ b/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineConfFileParseException.java
@@ -0,0 +1,30 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.beeline.hs2connection;
+
+@SuppressWarnings("serial")
+public class BeelineConfFileParseException extends Exception {
+
+  BeelineConfFileParseException(String msg, Exception e) {
+    super(msg, e);
+  }
+
+  public BeelineConfFileParseException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/54dbd7fe/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineHS2ConnectionFileParseException.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineHS2ConnectionFileParseException.java b/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineHS2ConnectionFileParseException.java
index acddf82..2efa5f3 100644
--- a/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineHS2ConnectionFileParseException.java
+++ b/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineHS2ConnectionFileParseException.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hive.beeline.hs2connection;
 
-public class BeelineHS2ConnectionFileParseException extends Exception {
+public class BeelineHS2ConnectionFileParseException extends BeelineConfFileParseException {
   private static final long serialVersionUID = -748635913718300617L;
 
   BeelineHS2ConnectionFileParseException(String msg, Exception e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/54dbd7fe/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineSiteParseException.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineSiteParseException.java b/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineSiteParseException.java
new file mode 100644
index 0000000..f415e84
--- /dev/null
+++ b/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineSiteParseException.java
@@ -0,0 +1,30 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.beeline.hs2connection;
+
+@SuppressWarnings("serial")
+public class BeelineSiteParseException extends BeelineConfFileParseException {
+
+  public BeelineSiteParseException(String msg, Exception e) {
+    super(msg, e);
+  }
+
+  public BeelineSiteParseException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/54dbd7fe/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineSiteParser.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineSiteParser.java b/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineSiteParser.java
new file mode 100644
index 0000000..600d84e
--- /dev/null
+++ b/beeline/src/java/org/apache/hive/beeline/hs2connection/BeelineSiteParser.java
@@ -0,0 +1,145 @@
+/*
+ * 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.beeline.hs2connection;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This class implements HS2ConnectionFileParser for the named url configuration file named
+ * beeline-site.xml. The class looks for this file in ${user.home}/.beeline, ${HIVE_CONF_DIR} or
+ * /etc/conf/hive in that order and uses the first file found in the above locations.
+ */
+public class BeelineSiteParser implements HS2ConnectionFileParser {
+  /**
+   * Prefix string used for named jdbc uri configs
+   */
+  public static final String BEELINE_CONNECTION_NAMED_JDBC_URL_PREFIX = "beeline.hs2.jdbc.url.";
+  /**
+   * Property key used to provide the default named jdbc uri in the config file
+   */
+  public static final String DEFAULT_NAMED_JDBC_URL_PROPERTY_KEY = "default";
+
+  public static final String DEFAULT_BEELINE_SITE_FILE_NAME = "beeline-site.xml";
+  public static final String DEFAULT_BEELINE_SITE_LOCATION =
+      System.getProperty("user.home") + File.separator
+          + (System.getProperty("os.name").toLowerCase().indexOf("windows") != -1 ? "" : ".")
+          + "beeline" + File.separator;
+  public static final String ETC_HIVE_CONF_LOCATION =
+      File.separator + "etc" + File.separator + "hive" + File.separator + "conf";
+
+  private final List<String> locations = new ArrayList<>();
+  private static final Logger log = LoggerFactory.getLogger(BeelineSiteParser.class);
+
+  public BeelineSiteParser() {
+    // file locations to be searched in the correct order
+    locations.add(DEFAULT_BEELINE_SITE_LOCATION + DEFAULT_BEELINE_SITE_FILE_NAME);
+    if (System.getenv("HIVE_CONF_DIR") != null) {
+      locations
+          .add(System.getenv("HIVE_CONF_DIR") + File.separator + DEFAULT_BEELINE_SITE_FILE_NAME);
+    }
+    locations.add(ETC_HIVE_CONF_LOCATION + DEFAULT_BEELINE_SITE_FILE_NAME);
+  }
+
+  @VisibleForTesting
+  BeelineSiteParser(List<String> testLocations) {
+    if(testLocations == null) {
+      return;
+    }
+    locations.addAll(testLocations);
+  }
+
+  @Override
+  public Properties getConnectionProperties() throws BeelineSiteParseException {
+    Properties props = new Properties();
+    String fileLocation = getFileLocation();
+    if (fileLocation == null) {
+      log.debug("Could not find Beeline configuration file: {}", DEFAULT_BEELINE_SITE_FILE_NAME);
+      return props;
+    }
+    log.info("Beeline configuration file at: {}", fileLocation);
+    // load the properties from config file
+    Configuration conf = new Configuration(false);
+    conf.addResource(new Path(new File(fileLocation).toURI()));
+    try {
+      for (Entry<String, String> kv : conf) {
+        String key = kv.getKey();
+        if (key.startsWith(BEELINE_CONNECTION_NAMED_JDBC_URL_PREFIX)) {
+          props.setProperty(key.substring(BEELINE_CONNECTION_NAMED_JDBC_URL_PREFIX.length()),
+              kv.getValue());
+        }
+      }
+    } catch (Exception e) {
+      throw new BeelineSiteParseException(e.getMessage(), e);
+    }
+    return props;
+  }
+
+  public Properties getConnectionProperties(String propertyValue) throws BeelineSiteParseException {
+    Properties props = new Properties();
+    String fileLocation = getFileLocation();
+    if (fileLocation == null) {
+      log.debug("Could not find Beeline configuration file: {}", DEFAULT_BEELINE_SITE_FILE_NAME);
+      return props;
+    }
+    log.info("Beeline configuration file at: {}", fileLocation);
+    // load the properties from config file
+    Configuration conf = new Configuration(false);
+    conf.addResource(new Path(new File(fileLocation).toURI()));
+    try {
+      for (Entry<String, String> kv : conf) {
+        String key = kv.getKey();
+        if (key.startsWith(BEELINE_CONNECTION_NAMED_JDBC_URL_PREFIX)
+            && (propertyValue.equalsIgnoreCase(kv.getValue()))) {
+          props.setProperty(key.substring(BEELINE_CONNECTION_NAMED_JDBC_URL_PREFIX.length()),
+              kv.getValue());
+        }
+      }
+    } catch (Exception e) {
+      throw new BeelineSiteParseException(e.getMessage(), e);
+    }
+    return props;
+  }
+
+  @Override
+  public boolean configExists() {
+    return (getFileLocation() != null);
+  }
+  /*
+   * This method looks in locations specified above and returns the first location where the file
+   * exists. If the file does not exist in any one of the locations it returns null
+   */
+  String getFileLocation() {
+    for (String location : locations) {
+      if (new File(location).exists()) {
+        return location;
+      }
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/54dbd7fe/beeline/src/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileParser.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileParser.java b/beeline/src/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileParser.java
index b769e85..3e169eb 100644
--- a/beeline/src/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileParser.java
+++ b/beeline/src/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileParser.java
@@ -79,7 +79,7 @@ public interface HS2ConnectionFileParser {
    * object if the connection configuration is not found
    * @throws BeelineHS2ConnectionFileParseException if there is invalid key with appropriate message
    */
-  Properties getConnectionProperties() throws BeelineHS2ConnectionFileParseException;
+  Properties getConnectionProperties() throws BeelineConfFileParseException;
   /**
    *
    * @return returns true if the configuration exists else returns false

http://git-wip-us.apache.org/repos/asf/hive/blob/54dbd7fe/beeline/src/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileUtils.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileUtils.java b/beeline/src/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileUtils.java
index f635b40..7c064cf 100644
--- a/beeline/src/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileUtils.java
+++ b/beeline/src/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileUtils.java
@@ -21,7 +21,12 @@ import java.io.File;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Properties;
+import java.util.Set;
+
+import org.apache.hive.jdbc.Utils.JdbcConnectionParams;
 
 public class HS2ConnectionFileUtils {
 
@@ -57,6 +62,7 @@ public class HS2ConnectionFileUtils {
       hiveVarProperties = extractHiveVariables(
           (String) props.remove(HS2ConnectionFileParser.HIVE_VAR_PROPERTY_KEY), false);
     }
+
     StringBuilder urlSb = new StringBuilder();
     urlSb.append(urlPrefix.trim());
     urlSb.append(hosts.trim());
@@ -116,4 +122,129 @@ public class HS2ConnectionFileUtils {
       hivePropertiesList.append(keyValue[1].trim());
     }
   }
+
+  public static String getNamedUrl(Properties userNamedConnectionURLs, String urlName)
+      throws BeelineSiteParseException {
+    String jdbcURL = null;
+    if ((urlName != null) && !urlName.isEmpty()) {
+      // Try to read the given named url from the connection configuration file
+      jdbcURL = userNamedConnectionURLs.getProperty(urlName);
+      if (jdbcURL == null) {
+        throw new BeelineSiteParseException(
+            "The named url: " + urlName + " is not specified in the connection configuration file: "
+                + BeelineSiteParser.DEFAULT_BEELINE_SITE_FILE_NAME);
+      }
+      return jdbcURL;
+    } else {
+      // Try to read the default named url from the connection configuration file
+      String defaultURLName = userNamedConnectionURLs
+          .getProperty(BeelineSiteParser.DEFAULT_NAMED_JDBC_URL_PROPERTY_KEY);
+      jdbcURL = userNamedConnectionURLs.getProperty(defaultURLName);
+      if (jdbcURL != null) {
+        return jdbcURL;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Merge the connection properties read from beeline-hs2-connection.xml with the
+   * JdbcConnectionParams extracted from the jdbc url specified in beeline.xml
+   *
+   * @param userConnectionProperties
+   * @param jdbcConnectionParams
+   * @return
+   * @throws BeelineHS2ConnectionFileParseException
+   */
+  public static Properties mergeUserConnectionPropertiesAndBeelineSite(
+      Properties userConnectionProperties, JdbcConnectionParams jdbcConnectionParams)
+      throws BeelineHS2ConnectionFileParseException {
+    Properties mergedConnectionProperties = new Properties();
+
+    userConnectionProperties.setProperty(HS2ConnectionFileParser.URL_PREFIX_PROPERTY_KEY,
+        "jdbc:hive2://");
+
+    // Host
+    String host =
+        getMergedProperty(userConnectionProperties, jdbcConnectionParams.getSuppliedURLAuthority(),
+            HS2ConnectionFileParser.HOST_PROPERTY_KEY, null);
+    if (host != null) {
+      mergedConnectionProperties.setProperty(HS2ConnectionFileParser.HOST_PROPERTY_KEY, host);
+    }
+
+    // Database
+    String defaultDB = getMergedProperty(userConnectionProperties, jdbcConnectionParams.getDbName(),
+        HS2ConnectionFileParser.DEFAULT_DB_PROPERTY_KEY, "default");
+    mergedConnectionProperties.setProperty(HS2ConnectionFileParser.DEFAULT_DB_PROPERTY_KEY,
+        defaultDB);
+
+    // hive conf
+    String hiveConfProperties = getMergedPropertiesString(userConnectionProperties,
+        HS2ConnectionFileParser.HIVE_CONF_PROPERTY_KEY, jdbcConnectionParams.getHiveConfs());
+    if (!hiveConfProperties.isEmpty()) {
+      mergedConnectionProperties.setProperty(HS2ConnectionFileParser.HIVE_CONF_PROPERTY_KEY,
+          hiveConfProperties);
+    }
+
+    // hive vars
+    String hiveVarProperties = getMergedPropertiesString(userConnectionProperties,
+        HS2ConnectionFileParser.HIVE_VAR_PROPERTY_KEY, jdbcConnectionParams.getHiveVars());
+    if (!hiveVarProperties.isEmpty()) {
+      mergedConnectionProperties.setProperty(HS2ConnectionFileParser.HIVE_VAR_PROPERTY_KEY,
+          hiveVarProperties);
+    }
+
+    // session vars
+    for (Map.Entry<String, String> entry : jdbcConnectionParams.getSessionVars().entrySet()) {
+      mergedConnectionProperties.setProperty(entry.getKey(), entry.getValue());
+    }
+    if (userConnectionProperties != null) {
+      for (String propName : userConnectionProperties.stringPropertyNames()) {
+        mergedConnectionProperties.setProperty(propName,
+            userConnectionProperties.getProperty(propName));
+      }
+    }
+    return mergedConnectionProperties;
+  }
+
+  private static String getMergedProperty(Properties userConnectionProperties,
+      String valueFromJdbcUri, String propertyKey, String defaultValue) {
+    String value = null;
+    if (userConnectionProperties != null) {
+      value = (String) userConnectionProperties.remove(propertyKey);
+    }
+    if (value == null || value.isEmpty()) {
+      value = valueFromJdbcUri;
+    }
+    if (value == null || value.isEmpty()) {
+      value = defaultValue;
+    }
+    return value;
+  }
+
+  private static String getMergedPropertiesString(Properties userConnectionProperties,
+      String propertyKey, Map<String, String> propertiesFromJdbcConnParams)
+      throws BeelineHS2ConnectionFileParseException {
+    String properties = "";
+    if ((userConnectionProperties != null) && (userConnectionProperties.containsKey(propertyKey))) {
+      properties =
+          extractHiveVariables((String) userConnectionProperties.remove(propertyKey), true);
+    }
+    String propertiesFromJdbcUri = "";
+    for (Map.Entry<String, String> entry : propertiesFromJdbcConnParams.entrySet()) {
+      if (!properties.contains(entry.getKey())) {
+        if (!propertiesFromJdbcUri.isEmpty()) {
+          propertiesFromJdbcUri = propertiesFromJdbcUri + ",";
+        }
+        propertiesFromJdbcUri = propertiesFromJdbcUri + entry.getKey() + "=" + entry.getValue();
+      }
+    }
+    if (!propertiesFromJdbcUri.isEmpty()) {
+      if (!properties.isEmpty()) {
+        properties = properties + ",";
+      }
+      properties = properties + propertiesFromJdbcUri;
+    }
+    return properties;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/54dbd7fe/beeline/src/java/org/apache/hive/beeline/hs2connection/UserHS2ConnectionFileParser.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/hs2connection/UserHS2ConnectionFileParser.java b/beeline/src/java/org/apache/hive/beeline/hs2connection/UserHS2ConnectionFileParser.java
index 2801ebe..9d45daf 100644
--- a/beeline/src/java/org/apache/hive/beeline/hs2connection/UserHS2ConnectionFileParser.java
+++ b/beeline/src/java/org/apache/hive/beeline/hs2connection/UserHS2ConnectionFileParser.java
@@ -86,9 +86,6 @@ public class UserHS2ConnectionFileParser implements HS2ConnectionFileParser {
         if (key.startsWith(BEELINE_CONNECTION_PROPERTY_PREFIX)) {
           props.setProperty(key.substring(BEELINE_CONNECTION_PROPERTY_PREFIX.length()),
               kv.getValue());
-        } else {
-          log.warn("Ignoring " + key + " since it does not start with "
-              + BEELINE_CONNECTION_PROPERTY_PREFIX);
         }
       }
     } catch (Exception ex) {

http://git-wip-us.apache.org/repos/asf/hive/blob/54dbd7fe/beeline/src/main/resources/BeeLine.properties
----------------------------------------------------------------------
diff --git a/beeline/src/main/resources/BeeLine.properties b/beeline/src/main/resources/BeeLine.properties
index 6fca953..c41b3ed 100644
--- a/beeline/src/main/resources/BeeLine.properties
+++ b/beeline/src/main/resources/BeeLine.properties
@@ -157,6 +157,9 @@ interrupt-ctrl-c: Interrupting... Please be patient this may take some time.
 
 cmd-usage: Usage: java org.apache.hive.cli.beeline.BeeLine \n \
 \  -u <database url>               the JDBC URL to connect to\n \
+\  -c <named url>                  the named JDBC URL to connect to,\n \
+\                                  which should be present in beeline-site.xml\n \
+\                                  as the value of beeline.hs2.jdbc.url.<namedUrl>\n \
 \  -r                              reconnect to last saved connect url (in conjunction with !save)\n \
 \  -n <username>                   the username to connect as\n \
 \  -p <password>                   the password to connect as\n \

http://git-wip-us.apache.org/repos/asf/hive/blob/54dbd7fe/beeline/src/test/org/apache/hive/beeline/hs2connection/TestUserHS2ConnectionFileParser.java
----------------------------------------------------------------------
diff --git a/beeline/src/test/org/apache/hive/beeline/hs2connection/TestUserHS2ConnectionFileParser.java b/beeline/src/test/org/apache/hive/beeline/hs2connection/TestUserHS2ConnectionFileParser.java
index 1d17887..f5923d1 100644
--- a/beeline/src/test/org/apache/hive/beeline/hs2connection/TestUserHS2ConnectionFileParser.java
+++ b/beeline/src/test/org/apache/hive/beeline/hs2connection/TestUserHS2ConnectionFileParser.java
@@ -177,9 +177,7 @@ public class TestUserHS2ConnectionFileParser {
     testLocations.add(path);
     UserHS2ConnectionFileParser testHS2ConfigManager =
         new UserHS2ConnectionFileParser(testLocations);
-
-    String url = HS2ConnectionFileUtils.getUrl(testHS2ConfigManager.getConnectionProperties());
-    return url;
+    return HS2ConnectionFileUtils.getUrl(testHS2ConfigManager.getConnectionProperties());
   }
 
   private void createNewFile(final String path) throws Exception {

http://git-wip-us.apache.org/repos/asf/hive/blob/54dbd7fe/itests/hive-unit/src/test/java/org/apache/hive/beeline/hs2connection/BeelineWithHS2ConnectionFileTestBase.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/hs2connection/BeelineWithHS2ConnectionFileTestBase.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/hs2connection/BeelineWithHS2ConnectionFileTestBase.java
index 3da31ad..2ed631a 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/hs2connection/BeelineWithHS2ConnectionFileTestBase.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/hs2connection/BeelineWithHS2ConnectionFileTestBase.java
@@ -89,7 +89,7 @@ public abstract class BeelineWithHS2ConnectionFileTestBase {
     }
 
     @Override
-    public HS2ConnectionFileParser getUserHS2ConnFileParser() {
+    public UserHS2ConnectionFileParser getUserHS2ConnFileParser() {
       return testHs2ConfigFileManager;
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/54dbd7fe/jdbc/src/java/org/apache/hive/jdbc/Utils.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/Utils.java b/jdbc/src/java/org/apache/hive/jdbc/Utils.java
index 6d7787d..42b3975 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/Utils.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/Utils.java
@@ -51,6 +51,10 @@ public class Utils {
     */
   static final String DEFAULT_PORT = "10000";
 
+  // To parse the intermediate URI as a Java URI, we'll give a dummy authority(dummyhost:00000).
+  // Later, we'll substitute the dummy authority for a resolved authority.
+  static final String dummyAuthorityString = "dummyhost:00000";
+
   /**
    * Hive's default database name
    */
@@ -161,7 +165,7 @@ public class Utils {
     private Map<String,String> hiveVars = new LinkedHashMap<String,String>();
     private Map<String,String> sessionVars = new LinkedHashMap<String,String>();
     private boolean isEmbeddedMode = false;
-    private String[] authorityList;
+    private String suppliedURLAuthority;
     private String zooKeeperEnsemble = null;
     private String currentHostZnodePath;
     private final List<String> rejectedHostZnodePaths = new ArrayList<String>();
@@ -178,7 +182,7 @@ public class Utils {
       this.hiveVars.putAll(params.hiveVars);
       this.sessionVars.putAll(params.sessionVars);
       this.isEmbeddedMode = params.isEmbeddedMode;
-      this.authorityList = params.authorityList;
+      this.suppliedURLAuthority = params.suppliedURLAuthority;
       this.zooKeeperEnsemble = params.zooKeeperEnsemble;
       this.currentHostZnodePath = params.currentHostZnodePath;
       this.rejectedHostZnodePaths.addAll(rejectedHostZnodePaths);
@@ -216,8 +220,8 @@ public class Utils {
       return sessionVars;
     }
 
-    public String[] getAuthorityList() {
-      return authorityList;
+    public String getSuppliedURLAuthority() {
+      return suppliedURLAuthority;
     }
 
     public String getZooKeeperEnsemble() {
@@ -264,8 +268,8 @@ public class Utils {
       this.sessionVars = sessionVars;
     }
 
-    public void setSuppliedAuthorityList(String[] authorityList) {
-      this.authorityList = authorityList;
+    public void setSuppliedURLAuthority(String suppliedURLAuthority) {
+      this.suppliedURLAuthority = suppliedURLAuthority;
     }
 
     public void setZooKeeperEnsemble(String zooKeeperEnsemble) {
@@ -323,10 +327,28 @@ public class Utils {
    * @return
    * @throws SQLException
    */
-  static JdbcConnectionParams parseURL(String uri, Properties info) throws JdbcUriParseException,
-      SQLException, ZooKeeperHiveClientException {
-    JdbcConnectionParams connParams = new JdbcConnectionParams();
+  public static JdbcConnectionParams parseURL(String uri, Properties info)
+      throws JdbcUriParseException, SQLException, ZooKeeperHiveClientException {
+    JdbcConnectionParams connParams = extractURLComponents(uri, info);
+    if (ZooKeeperHiveClientHelper.isZkDynamicDiscoveryMode(connParams.getSessionVars())) {
+      configureConnParamsFromZooKeeper(connParams);
+    }
+    handleAllDeprecations(connParams);
+    return connParams;
+  }
 
+  /**
+   * This method handles the base parsing of the given jdbc uri. Some of JdbcConnectionParams
+   * returned from this method are updated if ZooKeeper is used for service discovery
+   *
+   * @param uri
+   * @param info
+   * @return
+   * @throws JdbcUriParseException
+   */
+  public static JdbcConnectionParams extractURLComponents(String uri, Properties info)
+      throws JdbcUriParseException {
+    JdbcConnectionParams connParams = new JdbcConnectionParams();
     if (!uri.startsWith(URL_PREFIX)) {
       throw new JdbcUriParseException("Bad URL format: Missing prefix " + URL_PREFIX);
     }
@@ -342,19 +364,14 @@ public class Utils {
     // configured on HiveServer2 (like: host1:port1,host2:port2,host3:port3)
     // We'll extract the authorities (host:port combo) from the URI, extract session vars, hive
     // confs & hive vars by parsing it as a Java URI.
-    // To parse the intermediate URI as a Java URI, we'll give a dummy authority(dummy:00000).
-    // Later, we'll substitute the dummy authority for a resolved authority.
-    String dummyAuthorityString = "dummyhost:00000";
-    String suppliedAuthorities = getAuthorities(uri, connParams);
-    if ((suppliedAuthorities == null) || (suppliedAuthorities.isEmpty())) {
+    String authorityFromClientJdbcURL = getAuthorityFromJdbcURL(uri);
+    if ((authorityFromClientJdbcURL == null) || (authorityFromClientJdbcURL.isEmpty())) {
       // Given uri of the form:
       // jdbc:hive2:///dbName;sess_var_list?hive_conf_list#hive_var_list
       connParams.setEmbeddedMode(true);
     } else {
-      LOG.info("Supplied authorities: " + suppliedAuthorities);
-      String[] authorityList = suppliedAuthorities.split(",");
-      connParams.setSuppliedAuthorityList(authorityList);
-      uri = uri.replace(suppliedAuthorities, dummyAuthorityString);
+      connParams.setSuppliedURLAuthority(authorityFromClientJdbcURL);
+      uri = uri.replace(authorityFromClientJdbcURL, dummyAuthorityString);
     }
 
     // Now parse the connection uri with dummy authority
@@ -379,9 +396,10 @@ public class Utils {
         if (sessVars != null) {
           Matcher sessMatcher = pattern.matcher(sessVars);
           while (sessMatcher.find()) {
-            if (connParams.getSessionVars().put(sessMatcher.group(1), sessMatcher.group(2)) != null) {
-              throw new JdbcUriParseException("Bad URL format: Multiple values for property "
-                  + sessMatcher.group(1));
+            if (connParams.getSessionVars().put(sessMatcher.group(1),
+                sessMatcher.group(2)) != null) {
+              throw new JdbcUriParseException(
+                  "Bad URL format: Multiple values for property " + sessMatcher.group(1));
             }
           }
         }
@@ -414,32 +432,93 @@ public class Utils {
       if ((kv.getKey() instanceof String)) {
         String key = (String) kv.getKey();
         if (key.startsWith(JdbcConnectionParams.HIVE_VAR_PREFIX)) {
-          connParams.getHiveVars().put(
-              key.substring(JdbcConnectionParams.HIVE_VAR_PREFIX.length()), info.getProperty(key));
+          connParams.getHiveVars().put(key.substring(JdbcConnectionParams.HIVE_VAR_PREFIX.length()),
+              info.getProperty(key));
         } else if (key.startsWith(JdbcConnectionParams.HIVE_CONF_PREFIX)) {
           connParams.getHiveConfs().put(
               key.substring(JdbcConnectionParams.HIVE_CONF_PREFIX.length()), info.getProperty(key));
         }
       }
     }
+
     // Extract user/password from JDBC connection properties if its not supplied
     // in the connection URL
     if (!connParams.getSessionVars().containsKey(JdbcConnectionParams.AUTH_USER)) {
-        if (info.containsKey(JdbcConnectionParams.AUTH_USER)) {
-            connParams.getSessionVars().put(JdbcConnectionParams.AUTH_USER,
-              info.getProperty(JdbcConnectionParams.AUTH_USER));
-        }
-        if (info.containsKey(JdbcConnectionParams.AUTH_PASSWD)) {
-          connParams.getSessionVars().put(JdbcConnectionParams.AUTH_PASSWD,
-              info.getProperty(JdbcConnectionParams.AUTH_PASSWD));
-        }
+      if (info.containsKey(JdbcConnectionParams.AUTH_USER)) {
+        connParams.getSessionVars().put(JdbcConnectionParams.AUTH_USER,
+            info.getProperty(JdbcConnectionParams.AUTH_USER));
+      }
+      if (info.containsKey(JdbcConnectionParams.AUTH_PASSWD)) {
+        connParams.getSessionVars().put(JdbcConnectionParams.AUTH_PASSWD,
+            info.getProperty(JdbcConnectionParams.AUTH_PASSWD));
+      }
     }
 
     if (info.containsKey(JdbcConnectionParams.AUTH_TYPE)) {
       connParams.getSessionVars().put(JdbcConnectionParams.AUTH_TYPE,
           info.getProperty(JdbcConnectionParams.AUTH_TYPE));
     }
+    // Extract host, port
+    if (connParams.isEmbeddedMode()) {
+      // In case of embedded mode we were supplied with an empty authority.
+      // So we never substituted the authority with a dummy one.
+      connParams.setHost(jdbcURI.getHost());
+      connParams.setPort(jdbcURI.getPort());
+    } else {
+      String authorityStr = connParams.getSuppliedURLAuthority();
+      // If we're using ZooKeeper, the final host, port will be read from ZooKeeper
+      // (in a different method call). Therefore, we put back the original authority string
+      // (which basically is the ZooKeeper ensemble) back in the uri
+      if (ZooKeeperHiveClientHelper.isZkDynamicDiscoveryMode(connParams.getSessionVars())) {
+        uri = uri.replace(dummyAuthorityString, authorityStr);
+        // Set ZooKeeper ensemble in connParams for later use
+        connParams.setZooKeeperEnsemble(authorityStr);
+      } else {
+        URI jdbcBaseURI = URI.create(URI_HIVE_PREFIX + "//" + authorityStr);
+        // 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 -
+        // jdbc:hive2://localhost:10000;principal=hive/HiveServer2Host@YOUR-REALM.COM
+        if (jdbcBaseURI.getAuthority() != null) {
+          String host = jdbcBaseURI.getHost();
+          int port = jdbcBaseURI.getPort();
+          if (host == null) {
+            throw new JdbcUriParseException(
+                "Bad URL format. Hostname not found " + " in authority part of the url: "
+                    + jdbcBaseURI.getAuthority() + ". Are you missing a '/' after the hostname ?");
+          }
+          // Set the port to default value; we do support jdbc url like:
+          // jdbc:hive2://localhost/db
+          if (port <= 0) {
+            port = Integer.parseInt(Utils.DEFAULT_PORT);
+          }
+          connParams.setHost(jdbcBaseURI.getHost());
+          connParams.setPort(jdbcBaseURI.getPort());
+        }
+        // We check for invalid host, port while configuring connParams with configureConnParams()
+        authorityStr = connParams.getHost() + ":" + connParams.getPort();
+        LOG.debug("Resolved authority: " + authorityStr);
+        uri = uri.replace(dummyAuthorityString, authorityStr);
+      }
+    }
+    connParams.setJdbcUriString(uri);
+    return connParams;
+  }
 
+  // Configure using ZooKeeper
+  static void configureConnParamsFromZooKeeper(JdbcConnectionParams connParams)
+      throws ZooKeeperHiveClientException, JdbcUriParseException {
+    ZooKeeperHiveClientHelper.configureConnParams(connParams);
+    String authorityStr = connParams.getHost() + ":" + connParams.getPort();
+    LOG.debug("Resolved authority: " + authorityStr);
+    String jdbcUriString = connParams.getJdbcUriString();
+    // Replace ZooKeeper ensemble from the authority component of the JDBC Uri provided by the
+    // client, by the host:port of the resolved server instance we will connect to
+    connParams.setJdbcUriString(
+        jdbcUriString.replace(getAuthorityFromJdbcURL(jdbcUriString), authorityStr));
+  }
+
+  private static void handleAllDeprecations(JdbcConnectionParams connParams) {
     // Handle all deprecations here:
     String newUsage;
     String usageUrlBase = "jdbc:hive2://<host>:<port>/dbName;";
@@ -458,23 +537,6 @@ public class Utils {
     newUsage = usageUrlBase + JdbcConnectionParams.HTTP_PATH + "=<http_path_value>";
     handleParamDeprecation(connParams.getHiveConfs(), connParams.getSessionVars(),
         JdbcConnectionParams.HTTP_PATH_DEPRECATED, JdbcConnectionParams.HTTP_PATH, newUsage);
-    // Extract host, port
-    if (connParams.isEmbeddedMode()) {
-      // In case of embedded mode we were supplied with an empty authority.
-      // So we never substituted the authority with a dummy one.
-      connParams.setHost(jdbcURI.getHost());
-      connParams.setPort(jdbcURI.getPort());
-    } else {
-      // Configure host, port and params from ZooKeeper if used,
-      // and substitute the dummy authority with a resolved one
-      configureConnParams(connParams);
-      // We check for invalid host, port while configuring connParams with configureConnParams()
-      String authorityStr = connParams.getHost() + ":" + connParams.getPort();
-      LOG.info("Resolved authority: " + authorityStr);
-      uri = uri.replace(dummyAuthorityString, authorityStr);
-      connParams.setJdbcUriString(uri);
-    }
-    return connParams;
   }
 
   /**
@@ -506,15 +568,13 @@ public class Utils {
    * @return
    * @throws JdbcUriParseException
    */
-  private static String getAuthorities(String uri, JdbcConnectionParams connParams)
-      throws JdbcUriParseException {
+  private static String getAuthorityFromJdbcURL(String uri) throws JdbcUriParseException {
     String authorities;
     /**
      * For a jdbc uri like:
-     * jdbc:hive2://<host1>:<port1>,<host2>:<port2>/dbName;sess_var_list?conf_list#var_list
-     * Extract the uri host:port list starting after "jdbc:hive2://",
-     * till the 1st "/" or "?" or "#" whichever comes first & in the given order
-     * Examples:
+     * jdbc:hive2://<host1>:<port1>,<host2>:<port2>/dbName;sess_var_list?conf_list#var_list Extract
+     * the uri host:port list starting after "jdbc:hive2://", till the 1st "/" or "?" or "#"
+     * whichever comes first & in the given order Examples:
      * jdbc:hive2://host1:port1,host2:port2,host3:port3/db;k1=v1?k2=v2#k3=v3
      * jdbc:hive2://host1:port1,host2:port2,host3:port3/;k1=v1?k2=v2#k3=v3
      * jdbc:hive2://host1:port1,host2:port2,host3:port3?k2=v2#k3=v3
@@ -537,39 +597,6 @@ public class Utils {
     return authorities;
   }
 
-  private static void configureConnParams(JdbcConnectionParams connParams)
-      throws JdbcUriParseException, ZooKeeperHiveClientException {
-    if (ZooKeeperHiveClientHelper.isZkDynamicDiscoveryMode(connParams.getSessionVars())) {
-      // Set ZooKeeper ensemble in connParams for later use
-      connParams.setZooKeeperEnsemble(joinStringArray(connParams.getAuthorityList(), ","));
-      // Configure using ZooKeeper
-      ZooKeeperHiveClientHelper.configureConnParams(connParams);
-    } else {
-      String authority = connParams.getAuthorityList()[0];
-      URI jdbcURI = URI.create(URI_HIVE_PREFIX + "//" + authority);
-      // 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 -
-      // jdbc:hive2://localhost:10000;principal=hive/HiveServer2Host@YOUR-REALM.COM
-      if (jdbcURI.getAuthority() != null) {
-        String host = jdbcURI.getHost();
-        int port = jdbcURI.getPort();
-        if (host == null) {
-          throw new JdbcUriParseException("Bad URL format. Hostname not found "
-              + " in authority part of the url: " + jdbcURI.getAuthority()
-              + ". Are you missing a '/' after the hostname ?");
-        }
-        // Set the port to default value; we do support jdbc url like:
-        // jdbc:hive2://localhost/db
-        if (port <= 0) {
-          port = Integer.parseInt(Utils.DEFAULT_PORT);
-        }
-        connParams.setHost(jdbcURI.getHost());
-        connParams.setPort(jdbcURI.getPort());
-      }
-    }
-  }
-
   /**
    * Read the next server coordinates (host:port combo) from ZooKeeper. Ignore the znodes already
    * explored. Also update the host, port, jdbcUriString and other configs published by the server.
@@ -596,17 +623,6 @@ public class Utils {
     return true;
   }
 
-  private static String joinStringArray(String[] stringArray, String seperator) {
-    StringBuilder stringBuilder = new StringBuilder();
-    for (int cur = 0, end = stringArray.length; cur < end; cur++) {
-      if (cur > 0) {
-        stringBuilder.append(seperator);
-      }
-      stringBuilder.append(stringArray[cur]);
-    }
-    return stringBuilder.toString();
-  }
-
   /**
    * Takes a version string delimited by '.' and '-' characters
    * and returns a partial version.