diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java index 1d468eb..8b843f0 100644 --- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java +++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java @@ -826,8 +826,10 @@ public int begin(String[] args, InputStream inputStream) throws IOException { } else { int code = initArgsFromCliVars(args); defaultConnect(false); - if (code != 0) + if (code != 0){ return code; + } + getOpts().updateBeeLineOptsFromConf(); } if (getOpts().getScriptFile() != null) { diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java b/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java index c1ec82a..44da283 100644 --- a/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java +++ b/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java @@ -42,6 +42,7 @@ import jline.TerminalFactory; import jline.console.completer.Completer; import jline.console.completer.StringsCompleter; +import org.apache.hadoop.hive.conf.HiveConf; class BeeLineOpts implements Completer { public static final int DEFAULT_MAX_WIDTH = 80; @@ -219,6 +220,15 @@ public void load(InputStream fin) throws IOException { loadProperties(p); } + /** + * Update the options after connection is established in CLI mode. + */ + public void updateBeeLineOptsFromConf() { + if (!beeLine.isBeeLine()) { + HiveConf conf = beeLine.getCommands().getHiveConf(true); + setForce(HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLIIGNOREERRORS)); + } + } public void loadProperties(Properties props) { for (Object element : props.keySet()) { diff --git a/beeline/src/java/org/apache/hive/beeline/Commands.java b/beeline/src/java/org/apache/hive/beeline/Commands.java index d490273..b07388a 100644 --- a/beeline/src/java/org/apache/hive/beeline/Commands.java +++ b/beeline/src/java/org/apache/hive/beeline/Commands.java @@ -721,7 +721,7 @@ public boolean sql(String line) { */ private Map getHiveVariables() { Map result = new HashMap<>(); - BufferedRows rows = getConfInternal(); + BufferedRows rows = getConfInternal(true); while (rows.hasNext()) { Rows.Row row = (Rows.Row) rows.next(); if (!row.isMeta) { @@ -731,27 +731,45 @@ public boolean sql(String line) { return result; } - private HiveConf getHiveConf() { + /** + * This method should only be used in CLI mode. + * + * @return the hive configuration from server side + */ + public HiveConf getHiveConf(boolean call) { HiveConf conf = new HiveConf(); - BufferedRows rows = getConfInternal(); - while (rows.hasNext()) { + BufferedRows rows = getConfInternal(call); + while (rows != null && rows.hasNext()) { addConf((Rows.Row) rows.next(), conf); } return conf; } - private BufferedRows getConfInternal() { + /** + * Use call statement to retrieve the configurations for substitution and sql for the substitution. + * + * @param call + * @return + */ + private BufferedRows getConfInternal(boolean call) { Statement stmnt = null; BufferedRows rows = null; try { - stmnt = beeLine.createStatement(); - boolean hasResults = stmnt.execute("set"); + boolean hasResults; + if (call) { + stmnt = beeLine.getDatabaseConnection().getConnection().prepareCall("set"); + hasResults = ((CallableStatement) stmnt).execute(); + } else { + stmnt = beeLine.createStatement(); + hasResults = stmnt.execute("set"); + } if (hasResults) { ResultSet rs = stmnt.getResultSet(); rows = new BufferedRows(beeLine, rs); } } catch (SQLException e) { beeLine.error(e); + } finally { if (stmnt != null) { try { stmnt.close(); @@ -802,7 +820,7 @@ private boolean sourceFile(String cmd) { String[] tokens = tokenizeCmd(cmd); String cmd_1 = getFirstCmd(cmd, tokens[0].length()); - cmd_1 = substituteVariables(getHiveConf(), cmd_1); + cmd_1 = substituteVariables(getHiveConf(false), cmd_1); File sourceFile = new File(cmd_1); if (!sourceFile.isFile()) { return false; @@ -1020,7 +1038,7 @@ public boolean sh(String line) { } line = line.substring("sh".length()).trim(); - line = substituteVariables(getHiveConf(), line.trim()); + line = substituteVariables(getHiveConf(false), line.trim()); try { ShellCmdExecutor executor = new ShellCmdExecutor(line, beeLine.getOutputStream(), diff --git a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java index ff8ab17..fa94c89 100644 --- a/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java +++ b/beeline/src/test/org/apache/hive/beeline/cli/TestHiveCli.java @@ -78,7 +78,8 @@ private void verifyCMD(String CMD, String keywords, OutputStream os, String[] op int retCode) { executeCMD(options, CMD, retCode); String output = os.toString(); - Assert.assertTrue(output.contains(keywords)); + Assert.assertTrue("The expected keyword doesn't occur in the output: " + output, + output.contains(keywords)); } @Test