Skip to content

Commit

Permalink
HIVE-18963: JDBC: Provide an option to simplify beeline usage by supp…
Browse files Browse the repository at this point in the history
…orting default and named URL for beeline (Vaibhav Gumashta reviewed by Vihang Karajgaonkar, Thejas Nair)
  • Loading branch information
Vaibhav Gumashta committed Apr 2, 2018
1 parent 733aecf commit 54dbd7f
Show file tree
Hide file tree
Showing 12 changed files with 536 additions and 136 deletions.
108 changes: 79 additions & 29 deletions beeline/src/java/org/apache/hive/beeline/BeeLine.java
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@
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;
Expand Down Expand Up @@ -75,9 +74,6 @@
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;
Expand All @@ -95,7 +91,9 @@
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;
Expand All @@ -104,6 +102,7 @@

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;

Expand Down Expand Up @@ -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")
Expand All @@ -329,57 +336,57 @@ 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'));

// -w (or) --password-file <file>
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'));

// -a <authType>
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
Expand All @@ -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
Expand All @@ -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());
}

Expand Down Expand Up @@ -789,7 +796,7 @@ int initArgs(String[] args) {
// 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;
Expand Down Expand Up @@ -1064,50 +1071,93 @@ public int begin(String[] args, InputStream inputStream) throws IOException {
* 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();
}

Expand Down
Original file line number Diff line number Diff line change
@@ -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);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down
Original file line number Diff line number Diff line change
@@ -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);
}
}
Loading

0 comments on commit 54dbd7f

Please sign in to comment.