diff --git beeline/pom.xml beeline/pom.xml
index a720d08..5503add 100644
--- beeline/pom.xml
+++ beeline/pom.xml
@@ -119,6 +119,11 @@
test
+ org.mockito
+ mockito-all
+ test
+
+
postgresql
postgresql
9.1-901.jdbc4
diff --git beeline/src/java/org/apache/hive/beeline/BeeLine.java beeline/src/java/org/apache/hive/beeline/BeeLine.java
index 66185f6..e0fa032 100644
--- beeline/src/java/org/apache/hive/beeline/BeeLine.java
+++ beeline/src/java/org/apache/hive/beeline/BeeLine.java
@@ -138,6 +138,7 @@
private String currentDatabase = null;
private History history;
+ // Indicates if this instance of beeline is running in compatibility mode, or beeline mode
private boolean isBeeLine = true;
private static final Options options = new Options();
@@ -1432,18 +1433,17 @@ String getPromptForCli() {
HiveConf conf = getCommands().getHiveConf(true);
prompt = conf.getVar(HiveConf.ConfVars.CLIPROMPT);
prompt = getCommands().substituteVariables(conf, prompt);
- return prompt + getFormattedDb(conf) + "> ";
+ return prompt + getFormattedDb() + "> ";
}
/**
* Retrieve the current database name string to display, based on the
* configuration value.
*
- * @param conf storing whether or not to show current db
* @return String to show user for current db value
*/
- String getFormattedDb(HiveConf conf) {
- if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLIPRINTCURRENTDB)) {
+ String getFormattedDb() {
+ if (!getOpts().getShowDbInPrompt()) {
return "";
}
String currDb = getCurrentDatabase();
@@ -1461,11 +1461,10 @@ String getPromptForBeeline() {
} else {
String printClosed = getDatabaseConnection().isClosed() ? " (closed)" : "";
return getPromptForBeeline(getDatabaseConnections().getIndex()
- + ": " + getDatabaseConnection().getUrl()) + printClosed + "> ";
+ + ": " + getDatabaseConnection().getUrl()) + printClosed + getFormattedDb() + "> ";
}
}
-
static String getPromptForBeeline(String url) {
if (url == null || url.length() == 0) {
url = "beeline";
@@ -1482,7 +1481,6 @@ static String getPromptForBeeline(String url) {
return url;
}
-
/**
* Try to obtain the current size of the specified {@link ResultSet} by jumping to the last row
* and getting the row number.
diff --git beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
index 5aaa385..ad191bb 100644
--- beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
+++ beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
@@ -66,6 +66,7 @@
private boolean silent = false;
private boolean color = false;
private boolean showHeader = true;
+ private boolean showDbInPrompt = false;
private int headerInterval = 100;
private boolean fastConnect = true;
private boolean autoCommit = false;
@@ -467,6 +468,25 @@ public boolean getShowHeader() {
}
}
+ public void setShowDbInPrompt(boolean showDbInPrompt) {
+ this.showDbInPrompt = showDbInPrompt;
+ }
+
+ /**
+ * In compatibility mode returns the beeline option provided by command line argument or config file
+ * In beeline mode returns the value of the hive.cli.print.current.db config variable
+ * @return Should the current db displayed in the prompt
+ */
+ public boolean getShowDbInPrompt() {
+ if (beeLine.isBeeLine()) {
+ return showDbInPrompt;
+ } else {
+ boolean showDatabase;
+ HiveConf conf = beeLine.getCommands().getHiveConf(true);
+ return HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLIPRINTCURRENTDB);
+ }
+ }
+
public void setHeaderInterval(int headerInterval) {
this.headerInterval = headerInterval;
}
diff --git beeline/src/java/org/apache/hive/beeline/ClientCommandHookFactory.java beeline/src/java/org/apache/hive/beeline/ClientCommandHookFactory.java
index c4d97bc..1ba194b 100644
--- beeline/src/java/org/apache/hive/beeline/ClientCommandHookFactory.java
+++ beeline/src/java/org/apache/hive/beeline/ClientCommandHookFactory.java
@@ -17,6 +17,11 @@
*/
package org.apache.hive.beeline;
+import org.apache.commons.collections.functors.ExceptionClosure;
+import org.apache.hive.jdbc.Utils;
+
+import java.sql.SQLException;
+
/**
* We need to update some client side information after executing some Hive Commands
*/
@@ -52,34 +57,96 @@ public UseCommandHook(String sql) {
@Override
public void postHook(BeeLine beeLine) {
- if (!beeLine.isBeeLine()) {
- // Handler multi-line sql
- String line = sql.replaceAll("\\s+", " ");
- String strs[] = line.split(" ");
- String dbName;
- if (strs == null || strs.length != 2) {
- // unable to parse the use command
+ // Handler multi-line sql
+ String line = sql.replaceAll("\\s+", " ");
+ String strs[] = line.split(" ");
+ String dbName;
+ if (strs == null || strs.length != 2) {
+ // unable to parse the use command
+ dbName = "";
+ } else {
+ dbName = strs[1];
+ }
+ beeLine.setCurrentDatabase(dbName);
+ }
+ }
+
+ public class ConnectCommandHook extends ClientHook {
+
+ public ConnectCommandHook(String sql) {
+ super(sql);
+ }
+
+ @Override
+ public void postHook(BeeLine beeLine) {
+ // Handler multi-line sql
+ String line = sql.replaceAll("\\s+", " ");
+ String strs[] = line.split(" ");
+ String dbName;
+ if (strs == null || strs.length < 1) {
+ // unable to parse the connect command
+ dbName = "";
+ } else {
+ try {
+ dbName = Utils.parseURL(strs[1]).getDbName();
+ } catch (Exception e) {
+ // unable to parse the connect command
dbName = "";
- } else {
- dbName = strs[1];
}
- beeLine.setCurrentDatabase(dbName);
}
+ beeLine.setCurrentDatabase(dbName);
}
}
- public ClientHook getHook(String cmdLine) {
- if (cmdLine.toLowerCase().startsWith("set")) {
- // Only set A = B command needs updating the configuration stored in client side.
- if (cmdLine.contains("=")) {
- return new SetCommandHook(cmdLine);
+ public class GoCommandHook extends ClientHook {
+
+ public GoCommandHook(String sql) {
+ super(sql);
+ }
+
+ @Override
+ public void postHook(BeeLine beeLine) {
+ String dbName = "";
+ try {
+ dbName = beeLine.getDatabaseConnection().getConnection().getSchema();
+ } catch (SQLException e) {
+ // unable to get the database, keep the current value
+ }
+ beeLine.setCurrentDatabase(dbName);
+ }
+ }
+
+ public ClientHook getHook(BeeLine beeLine, String cmdLine) {
+ if (!beeLine.isBeeLine()) {
+ // In compatibility mode we need to hook to set, and use
+ if (cmdLine.toLowerCase().startsWith("set")) {
+ // Only set A = B command needs updating the configuration stored in client side.
+ if (cmdLine.contains("=")) {
+ return new SetCommandHook(cmdLine);
+ } else {
+ return null;
+ }
+ } else if (cmdLine.toLowerCase().startsWith("use")) {
+ return new UseCommandHook(cmdLine);
} else {
return null;
}
- } else if (cmdLine.toLowerCase().startsWith("use")) {
- return new UseCommandHook(cmdLine);
} else {
- return null;
+ // In beeline mode we need to hook to use, connect, go, in case
+ // the ShowDbInPrompt is set, so the database name is needed
+ if (beeLine.getOpts().getShowDbInPrompt()) {
+ if (cmdLine.toLowerCase().startsWith("use")) {
+ return new UseCommandHook(cmdLine);
+ } else if (cmdLine.toLowerCase().startsWith("connect")) {
+ return new ConnectCommandHook(cmdLine);
+ } else if (cmdLine.toLowerCase().startsWith("go")) {
+ return new GoCommandHook(cmdLine);
+ } else {
+ return null;
+ }
+ } else {
+ return null;
+ }
}
}
}
diff --git beeline/src/java/org/apache/hive/beeline/Commands.java beeline/src/java/org/apache/hive/beeline/Commands.java
index 3a204c0..4b4abaa 100644
--- beeline/src/java/org/apache/hive/beeline/Commands.java
+++ beeline/src/java/org/apache/hive/beeline/Commands.java
@@ -952,10 +952,7 @@ private boolean executeInternal(String sql, boolean call) {
return false;
}
- ClientHook hook = null;
- if (!beeLine.isBeeLine()) {
- hook = ClientCommandHookFactory.get().getHook(sql);
- }
+ ClientHook hook = ClientCommandHookFactory.get().getHook(beeLine, sql);
try {
Statement stmnt = null;
diff --git beeline/src/java/org/apache/hive/beeline/ReflectiveCommandHandler.java beeline/src/java/org/apache/hive/beeline/ReflectiveCommandHandler.java
index 3b863ae..8205126 100644
--- beeline/src/java/org/apache/hive/beeline/ReflectiveCommandHandler.java
+++ beeline/src/java/org/apache/hive/beeline/ReflectiveCommandHandler.java
@@ -48,15 +48,23 @@ public ReflectiveCommandHandler(BeeLine beeLine, String[] cmds, Completer[] comp
public boolean execute(String line) {
lastException = null;
+ ClientHook hook = ClientCommandHookFactory.get().getHook(beeLine, line);
+
try {
Object ob = beeLine.getCommands().getClass().getMethod(getName(),
new Class[] {String.class})
.invoke(beeLine.getCommands(), new Object[] {line});
- return ob != null && ob instanceof Boolean
- && ((Boolean) ob).booleanValue();
+
+ boolean result = (ob != null && ob instanceof Boolean && ((Boolean) ob).booleanValue());
+
+ if (hook != null && result) {
+ hook.postHook(beeLine);
+ }
+
+ return result;
} catch (Throwable e) {
lastException = e;
return beeLine.error(e);
}
}
-}
\ No newline at end of file
+}
diff --git beeline/src/main/resources/BeeLine.properties beeline/src/main/resources/BeeLine.properties
index 7500df9..95b8fa1 100644
--- beeline/src/main/resources/BeeLine.properties
+++ beeline/src/main/resources/BeeLine.properties
@@ -167,6 +167,7 @@ cmd-usage: Usage: java org.apache.hive.cli.beeline.BeeLine \n \
\ --autoCommit=[true/false] enable/disable automatic transaction commit\n \
\ --verbose=[true/false] show verbose error messages and debug info\n \
\ --showWarnings=[true/false] display connection warnings\n \
+\ --showDbInPrompt=[true/false] display the current database name in the prompt\n \
\ --showNestedErrs=[true/false] display nested errors\n \
\ --numberFormat=[pattern] format numbers using DecimalFormat pattern\n \
\ --force=[true/false] continue running script even after errors\n \
diff --git beeline/src/test/org/apache/hive/beeline/TestBeelineArgParsing.java beeline/src/test/org/apache/hive/beeline/TestBeelineArgParsing.java
index ce1f538..a83f925 100644
--- beeline/src/test/org/apache/hive/beeline/TestBeelineArgParsing.java
+++ beeline/src/test/org/apache/hive/beeline/TestBeelineArgParsing.java
@@ -187,6 +187,25 @@ public void testBeelineOpts() throws Exception {
Assert.assertTrue(bl.getOpts().getTruncateTable());
}
+ @Test
+ public void testBeelineShowDbInPromptOptsDefault() throws Exception {
+ TestBeeline bl = new TestBeeline();
+ String args[] = new String[] { "-u", "url" };
+ Assert.assertEquals(0, bl.initArgs(args));
+ Assert.assertFalse(bl.getOpts().getShowDbInPrompt());
+ Assert.assertEquals("", bl.getFormattedDb());
+ }
+
+ @Test
+ public void testBeelineShowDbInPromptOptsTrue() throws Exception {
+ TestBeeline bl = new TestBeeline();
+ String args[] = new String[] { "-u", "url", "--showDbInPrompt=true" };
+ Assert.assertEquals(0, bl.initArgs(args));
+ Assert.assertTrue(bl.getOpts().getShowDbInPrompt());
+ Assert.assertEquals(" (default)", bl.getFormattedDb());
+ }
+
+
/**
* Test setting script file with -f option.
*/
diff --git beeline/src/test/org/apache/hive/beeline/TestClientCommandHookFactory.java beeline/src/test/org/apache/hive/beeline/TestClientCommandHookFactory.java
index c86de0a..4c9cafb 100644
--- beeline/src/test/org/apache/hive/beeline/TestClientCommandHookFactory.java
+++ beeline/src/test/org/apache/hive/beeline/TestClientCommandHookFactory.java
@@ -17,16 +17,110 @@
*/
package org.apache.hive.beeline;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.times;
+
import junit.framework.Assert;
+import org.junit.BeforeClass;
import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+import java.sql.Connection;
+import java.sql.SQLException;
public class TestClientCommandHookFactory {
+ public BeeLine setupMockData(boolean beeline, boolean showDbInPrompt) {
+ BeeLine mockBeeline = mock(BeeLine.class);
+ DatabaseConnection mockDatabaseConnection = mock(DatabaseConnection.class);
+ Connection mockConnection = mock(Connection.class);
+ try {
+ when(mockConnection.getSchema()).thenReturn("newDatabase");
+ when(mockDatabaseConnection.getConnection()).thenReturn(mockConnection);
+ } catch(SQLException sqlException) {
+ // We do mnot test this
+ }
+ when(mockBeeline.getDatabaseConnection()).thenReturn(mockDatabaseConnection);
+ BeeLineOpts mockBeeLineOpts = mock(BeeLineOpts.class);
+ when(mockBeeLineOpts.getShowDbInPrompt()).thenReturn(showDbInPrompt);
+ when(mockBeeline.getOpts()).thenReturn(mockBeeLineOpts);
+ when(mockBeeline.isBeeLine()).thenReturn(beeline);
+
+ return mockBeeline;
+ }
+ @BeforeClass
+ public static void setupMockData() {
+ }
+
@Test
- public void testGetHook() {
- Assert.assertNull(ClientCommandHookFactory.get().getHook("set a;"));
+ public void testGetHookCli() {
+ BeeLine beeLine = setupMockData(false, false);
+ Assert.assertNull(ClientCommandHookFactory.get().getHook(beeLine, "set a;"));
+ Assert.assertTrue(ClientCommandHookFactory.get()
+ .getHook(beeLine, "set a=b;") instanceof ClientCommandHookFactory.SetCommandHook);
+ Assert.assertTrue(ClientCommandHookFactory.get()
+ .getHook(beeLine, "USE a.b") instanceof ClientCommandHookFactory.UseCommandHook);
+ Assert.assertNull(ClientCommandHookFactory.get().getHook(beeLine, "coNNect a.b"));
+ Assert.assertNull(ClientCommandHookFactory.get().getHook(beeLine, "gO 1"));
+ Assert.assertNull(ClientCommandHookFactory.get().getHook(beeLine, "g"));
+ }
+
+ @Test
+ public void testGetHookBeeLineWithShowDbInPrompt() {
+ BeeLine beeLine = setupMockData(true, true);
+ Assert.assertNull(ClientCommandHookFactory.get().getHook(beeLine, "set a;"));
+ Assert.assertNull(ClientCommandHookFactory.get().getHook(beeLine, "set a=b;"));
+ Assert.assertTrue(ClientCommandHookFactory.get()
+ .getHook(beeLine, "USE a.b") instanceof ClientCommandHookFactory.UseCommandHook);
Assert.assertTrue(ClientCommandHookFactory.get()
- .getHook("set a=b;") instanceof ClientCommandHookFactory.SetCommandHook);
+ .getHook(beeLine, "coNNect a.b") instanceof ClientCommandHookFactory.ConnectCommandHook);
Assert.assertTrue(ClientCommandHookFactory.get()
- .getHook("USE a.b") instanceof ClientCommandHookFactory.UseCommandHook);
+ .getHook(beeLine, "gO 1") instanceof ClientCommandHookFactory.GoCommandHook);
+ Assert.assertNull(ClientCommandHookFactory.get().getHook(beeLine, "g"));
+ }
+
+ @Test
+ public void testGetHookBeeLineWithoutShowDbInPrompt() {
+ BeeLine beeLine = setupMockData(true, false);
+ Assert.assertNull(ClientCommandHookFactory.get().getHook(beeLine, "set a;"));
+ Assert.assertNull(ClientCommandHookFactory.get().getHook(beeLine, "set a=b;"));
+ Assert.assertNull(ClientCommandHookFactory.get().getHook(beeLine, "USE a.b"));
+ Assert.assertNull(ClientCommandHookFactory.get().getHook(beeLine, "coNNect a.b"));
+ Assert.assertNull(ClientCommandHookFactory.get().getHook(beeLine, "gO 1"));
+ Assert.assertNull(ClientCommandHookFactory.get().getHook(beeLine, "g"));
+ }
+
+ @Test
+ public void testUseHook() {
+ BeeLine beeLine = setupMockData(true, true);
+ ClientHook hook = ClientCommandHookFactory.get().getHook(beeLine, "USE newDatabase1");
+ Assert.assertTrue(hook instanceof ClientCommandHookFactory.UseCommandHook);
+ hook.postHook(beeLine);
+ ArgumentCaptor argument = ArgumentCaptor.forClass(String.class);
+ verify(beeLine).setCurrentDatabase(argument.capture());
+ Assert.assertEquals("newDatabase1", argument.getValue());
+ }
+
+ @Test
+ public void testConnectHook() {
+ BeeLine beeLine = setupMockData(true, true);
+ ClientHook hook = ClientCommandHookFactory.get().getHook(beeLine, "coNNect jdbc:hive2://localhost:10000/newDatabase2 a a");
+ Assert.assertTrue(hook instanceof ClientCommandHookFactory.ConnectCommandHook);
+ hook.postHook(beeLine);
+ ArgumentCaptor argument = ArgumentCaptor.forClass(String.class);
+ verify(beeLine).setCurrentDatabase(argument.capture());
+ Assert.assertEquals("newDatabase2", argument.getValue());
+ }
+
+ @Test
+ public void testGoHook() {
+ BeeLine beeLine = setupMockData(true, true);
+ ClientHook hook = ClientCommandHookFactory.get().getHook(beeLine, "go 1");
+ Assert.assertTrue(hook instanceof ClientCommandHookFactory.GoCommandHook);
+ hook.postHook(beeLine);
+ ArgumentCaptor argument = ArgumentCaptor.forClass(String.class);
+ verify(beeLine).setCurrentDatabase(argument.capture());
+ Assert.assertEquals("newDatabase", argument.getValue());
}
}
diff --git itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
index ecfeddb..becb995 100644
--- itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
+++ itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
@@ -277,7 +277,7 @@ public void testLastLineCmdInScriptFile() throws Throwable {
List argList = getBaseArgs(miniHS2.getBaseJdbcURL());
testScriptFile( SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
}
-
+
/**
* Test Beeline -hivevar option. User can specify --hivevar name=value on Beeline command line.
* In the script, user should be able to use it in the form of ${name}, which will be substituted with
@@ -863,4 +863,22 @@ public void testConnectionWithURLParams() throws Throwable {
testScriptFile( SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
}
+
+ /**
+ * Attempt to execute a simple script file with the usage of user & password variables in URL.
+ * Test for presence of an expected pattern
+ * in the output (stdout or stderr), fail if not found
+ * Print PASSED or FAILED
+ */
+ @Test
+ public void testShowDbInPrompt() throws Throwable {
+ final String EXPECTED_PATTERN = " (default)>";
+ List argList = new ArrayList();
+ argList.add("--showDbInPrompt");
+ argList.add("-u");
+ argList.add(miniHS2.getBaseJdbcURL() + ";user=hivetest;password=hive");
+ String SCRIPT_TEXT = "select current_user();";
+
+ testScriptFile( SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+ }
}
diff --git jdbc/src/java/org/apache/hive/jdbc/Utils.java jdbc/src/java/org/apache/hive/jdbc/Utils.java
index 7ea6309..3161566 100644
--- jdbc/src/java/org/apache/hive/jdbc/Utils.java
+++ jdbc/src/java/org/apache/hive/jdbc/Utils.java
@@ -266,6 +266,10 @@ static void verifySuccess(TStatus status, boolean withInfo) throws SQLException
throw new HiveSQLException(status);
}
+ public static JdbcConnectionParams parseURL(String uri) throws JdbcUriParseException,
+ SQLException, ZooKeeperHiveClientException {
+ return parseURL(uri, new Properties());
+ }
/**
* Parse JDBC connection URL
* The new format of the URL is: