exit is true.
@@ -1005,17 +930,9 @@ public class BeeLine implements Closeable {
// this method also initializes the consoleReader which is
// needed by initArgs for certain execution paths
ConsoleReader reader = initializeConsoleReader(inputStream);
- if (isBeeLine) {
- int code = initArgs(args);
- if (code != 0) {
- return code;
- }
- } else {
- int code = initArgsFromCliVars(args);
- if (code != 0 || exit) {
- return code;
- }
- defaultConnect(false);
+ int code = initArgs(args);
+ if (code != 0) {
+ return code;
}
if (getOpts().isHelpAsked()) {
@@ -1194,33 +1111,10 @@ public class BeeLine implements Closeable {
return ERRNO_OK;
}
- public int defaultConnect(boolean exitOnError) {
- if (embeddedConnect() != ERRNO_OK && exitOnError) {
- return ERRNO_OTHER;
- }
- if (connectDBInEmbededMode() != ERRNO_OK && exitOnError) {
- return ERRNO_OTHER;
- }
- return ERRNO_OK;
- }
-
private int executeFile(String fileName) {
InputStream fileStream = null;
try {
- if (!isBeeLine) {
- org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(fileName);
- FileSystem fs;
- HiveConf conf = getCommands().getHiveConf(true);
- if (!path.toUri().isAbsolute()) {
- fs = FileSystem.getLocal(conf);
- path = fs.makeQualified(path);
- } else {
- fs = FileSystem.get(path.toUri(), conf);
- }
- fileStream = fs.open(path);
- } else {
- fileStream = new FileInputStream(fileName);
- }
+ fileStream = new FileInputStream(fileName);
return execute(initializeConsoleReader(fileStream), !getOpts().getForce());
} catch (Throwable t) {
handleException(t);
@@ -1385,13 +1279,9 @@ public class BeeLine implements Closeable {
line = "!help";
}
- if (isBeeLine) {
- if (line.startsWith(COMMAND_PREFIX)) {
- // handle SQLLine command in beeline which starts with ! and does not end with ;
- return execCommandWithPrefix(line);
- } else {
- return commands.sql(line, getOpts().getEntireLineAsCommand());
- }
+ if (line.startsWith(COMMAND_PREFIX)) {
+ // handle SQLLine command in beeline which starts with ! and does not end with ;
+ return execCommandWithPrefix(line);
} else {
return commands.sql(line, getOpts().getEntireLineAsCommand());
}
@@ -1664,20 +1554,7 @@ public class BeeLine implements Closeable {
}
String getPrompt() {
- if (isBeeLine) {
- return getPromptForBeeline();
- } else {
- return getPromptForCli();
- }
- }
-
- String getPromptForCli() {
- String prompt;
- // read prompt configuration and substitute variables.
- HiveConf conf = getCommands().getHiveConf(true);
- prompt = conf.getVar(HiveConf.ConfVars.CLIPROMPT);
- prompt = getCommands().substituteVariables(conf, prompt);
- return prompt + getFormattedDb() + "> ";
+ return getPromptForBeeline();
}
/**
diff --git a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/BeeLineOpts.java b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/BeeLineOpts.java
index 2bd396432..4a37d8a38 100644
--- a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/BeeLineOpts.java
+++ b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/BeeLineOpts.java
@@ -45,7 +45,6 @@ import jline.TerminalFactory;
import jline.console.completer.Completer;
import jline.console.completer.StringsCompleter;
import jline.console.history.MemoryHistory;
-import org.apache.hadoop.hive.conf.HiveConf;
class BeeLineOpts implements Completer {
public static final int DEFAULT_MAX_WIDTH = 80;
@@ -89,8 +88,6 @@ class BeeLineOpts implements Completer {
int timeout = -1;
private String isolation = DEFAULT_ISOLATION_LEVEL;
private String outputFormat = "table";
- // This configuration is used only for client side configuration.
- private HiveConf conf;
private boolean trimScripts = true;
private boolean allowMultiLineCommand = true;
@@ -268,20 +265,6 @@ class BeeLineOpts implements Completer {
loadProperties(p);
}
- /** Update the options after connection is established in CLI mode. */
- public void updateBeeLineOptsFromConf() {
- if (!beeLine.isBeeLine()) {
- if (conf == null) {
- conf = beeLine.getCommands().getHiveConf(false);
- }
- setForce(HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLIIGNOREERRORS));
- }
- }
-
- public void setHiveConf(HiveConf conf) {
- this.conf = conf;
- }
-
public void loadProperties(Properties props) {
for (Object element : props.keySet()) {
String key = element.toString();
@@ -469,14 +452,7 @@ class BeeLineOpts implements Completer {
}
public boolean getShowHeader() {
- if (beeLine.isBeeLine()) {
- return showHeader;
- } else {
- boolean header;
- HiveConf conf = beeLine.getCommands().getHiveConf(true);
- header = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER);
- return header;
- }
+ return showHeader;
}
public void setEscapeCRLF(boolean escapeCRLF) {
@@ -484,14 +460,7 @@ class BeeLineOpts implements Completer {
}
public boolean getEscapeCRLF() {
- if (beeLine.isBeeLine()) {
- return escapeCRLF;
- } else {
- boolean flag;
- HiveConf conf = beeLine.getCommands().getHiveConf(true);
- flag = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_ESCAPE_CRLF);
- return flag;
- }
+ return escapeCRLF;
}
public void setShowDbInPrompt(boolean showDbInPrompt) {
@@ -505,12 +474,7 @@ class BeeLineOpts implements Completer {
* @return Should the current db displayed in the prompt
*/
public boolean getShowDbInPrompt() {
- if (beeLine.isBeeLine()) {
- return showDbInPrompt;
- } else {
- HiveConf conf = beeLine.getCommands().getHiveConf(true);
- return HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLIPRINTCURRENTDB);
- }
+ return showDbInPrompt;
}
public void setHeaderInterval(int headerInterval) {
@@ -651,11 +615,6 @@ class BeeLineOpts implements Completer {
this.delimiterForDSV = delimiterForDSV;
}
- @Ignore
- public HiveConf getConf() {
- return conf;
- }
-
public void setHelpAsked(boolean helpAsked) {
this.helpAsked = helpAsked;
}
diff --git a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/ClientCommandHookFactory.java b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/ClientCommandHookFactory.java
index cebcad1fe..beccbf1ec 100644
--- a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/ClientCommandHookFactory.java
+++ b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/ClientCommandHookFactory.java
@@ -30,20 +30,6 @@ public class ClientCommandHookFactory {
return instance;
}
- public class SetCommandHook extends ClientHook {
-
- public SetCommandHook(String sql) {
- super(sql);
- }
-
- @Override
- public void postHook(BeeLine beeLine) {
- if (!beeLine.isBeeLine()) {
- beeLine.getOpts().setHiveConf(beeLine.getCommands().getHiveConf(false));
- }
- }
- }
-
public class UseCommandHook extends ClientHook {
public UseCommandHook(String sql) {
@@ -112,36 +98,20 @@ public class ClientCommandHookFactory {
}
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")) {
+ // 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 {
- // 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;
- }
+ return null;
}
}
}
diff --git a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/Commands.java b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/Commands.java
index a1df5fb55..d414fca3e 100644
--- a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/Commands.java
+++ b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/Commands.java
@@ -52,14 +52,10 @@ import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.TreeSet;
-import org.apache.hadoop.hive.common.cli.ShellCmdExecutor;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveVariableSource;
-import org.apache.hadoop.hive.conf.SystemVariables;
-import org.apache.hadoop.hive.conf.VariableSubstitution;
import org.apache.hadoop.io.IOUtils;
+import org.apache.hive.beeline.common.cli.ShellCmdExecutor;
+import org.apache.hive.beeline.common.util.HiveStringUtils;
import org.apache.hive.beeline.logs.KyuubiBeelineInPlaceUpdateStream;
-import org.apache.hive.common.util.HiveStringUtils;
import org.apache.kyuubi.jdbc.hive.JdbcConnectionParams;
import org.apache.kyuubi.jdbc.hive.KyuubiStatement;
import org.apache.kyuubi.jdbc.hive.Utils;
@@ -754,29 +750,6 @@ public class Commands {
return result;
}
- /**
- * This method should only be used in CLI mode.
- *
- * @return the hive configuration from server side
- */
- public HiveConf getHiveConf(boolean call) {
- HiveConf hiveConf = beeLine.getOpts().getConf();
- if (hiveConf != null && call) {
- return hiveConf;
- } else {
- return getHiveConfHelper(call);
- }
- }
-
- public HiveConf getHiveConfHelper(boolean call) {
- HiveConf conf = new HiveConf();
- BufferedRows rows = getConfInternal(call);
- while (rows != null && rows.hasNext()) {
- addConf((Rows.Row) rows.next(), conf);
- }
- return conf;
- }
-
/**
* Use call statement to retrieve the configurations for substitution and sql for the
* substitution.
@@ -827,23 +800,6 @@ public class Commands {
return rows;
}
- private void addConf(Rows.Row r, HiveConf hiveConf) {
- if (r.isMeta) {
- return;
- }
- if (r.values == null || r.values[0] == null || r.values[0].isEmpty()) {
- return;
- }
- String val = r.values[0];
- if (r.values[0].startsWith(SystemVariables.SYSTEM_PREFIX)
- || r.values[0].startsWith(SystemVariables.ENV_PREFIX)) {
- return;
- } else {
- String[] kv = val.split("=", 2);
- if (kv.length == 2) hiveConf.set(kv[0], kv[1]);
- }
- }
-
/** Extract and clean up the first command in the input. */
private String getFirstCmd(String cmd, int length) {
return cmd.substring(length).trim();
@@ -863,7 +819,6 @@ public class Commands {
String[] tokens = tokenizeCmd(cmd);
String cmd_1 = getFirstCmd(cmd, tokens[0].length());
- cmd_1 = substituteVariables(getHiveConf(false), cmd_1);
File sourceFile = new File(cmd_1);
if (!sourceFile.isFile()) {
return false;
@@ -1123,21 +1078,6 @@ public class Commands {
return execute(line, false, entireLineAsCommand);
}
- public String substituteVariables(HiveConf conf, String line) {
- if (!beeLine.isBeeLine()) {
- // Substitution is only supported in non-beeline mode.
- return new VariableSubstitution(
- new HiveVariableSource() {
- @Override
- public MapOriginally copied from o.a.hadoop.util.StringUtils
+ */
+public class HiveStringUtils {
+
+ /**
+ * Strip comments from a sql statement, tracking when the statement contains a string literal.
+ *
+ * @param statement the input string
+ * @return a stripped statement
+ */
+ public static String removeComments(String statement) {
+ if (statement == null) {
+ return null;
+ }
+ IteratorRunnable
+ * @param priority priority of the shutdownHook.
+ */
+ public static void addShutdownHook(Runnable shutdownHook, int priority) {
+ if (priority < 0) {
+ throw new IllegalArgumentException("Priority should be greater than or equal to zero");
+ }
+ MGR.addShutdownHook(shutdownHook, priority);
+ }
+
+ /**
+ * Indicates if shutdown is in progress or not.
+ *
+ * @return TRUE if the shutdown is in progress, otherwise FALSE.
+ */
+ public static boolean isShutdownInProgress() {
+ return MGR.isShutdownInProgress();
+ }
+
+ /**
+ * Removes a shutdownHook.
+ *
+ * @param shutdownHook shutdownHook to remove.
+ * @return TRUE if the shutdownHook was registered and removed, FALSE otherwise (including when
+ * shutdownHook == null)
+ */
+ public static boolean removeShutdownHook(Runnable shutdownHook) {
+ if (shutdownHook == null) {
+ return false;
+ }
+ return MGR.removeShutdownHook(shutdownHook);
+ }
+
+ /** register file to delete-on-exit hook */
+ public static void deleteOnExit(File file) {
+ if (MGR.isShutdownInProgress()) {
+ LOG.warn("Shutdown in progress, cannot add a deleteOnExit");
+ }
+ DELETE_ON_EXIT_HOOK.deleteTargets.add(file);
+ }
+
+ /** deregister file from delete-on-exit hook */
+ public static void cancelDeleteOnExit(File file) {
+ if (MGR.isShutdownInProgress()) {
+ LOG.warn("Shutdown in progress, cannot cancel a deleteOnExit");
+ }
+ DELETE_ON_EXIT_HOOK.deleteTargets.remove(file);
+ }
+
+ @VisibleForTesting
+ static boolean isRegisteredToDeleteOnExit(File file) {
+ return DELETE_ON_EXIT_HOOK.deleteTargets.contains(file);
+ }
+
+ private static class DeleteOnExitHook implements Runnable {
+ private final Set