Skip to content

Commit

Permalink
[KYUUBI #6162] Cut out hive-common deps in beeline module
Browse files Browse the repository at this point in the history
# 🔍 Description
## Issue References 🔗

This PR removes the `hive-common` dep from the beeline module, and it is the final step of cutting out all Hive deps from the beeline module.

## Describe Your Solution 🔧

- Remove support of Hive CLI mode (a.k.a. `beelineMode == false`) on beeline.
- Copy ~7 classes (~600 lines) from `org.apache.hive:hive-common:3.1.3`
- Drop `org.apache.hive:hive-common:3.1.3` from beeline module

Note: Hive CLI mode removal is necessary, because it is useless for Kyuubi cases and seriously coupled with `HiveConf`

## Types of changes 🔖

- [ ] Bugfix (non-breaking change which fixes an issue)
- [ ] New feature (non-breaking change which adds functionality)
- [ ] Breaking change (fix or feature that would cause existing functionality to change)

## Test Plan 🧪

Pass GA, and manually tested.

```
roothadoop-master1:/opt/kyuubi# beeline
Connecting to jdbc:hive2://hadoop-master1.orb.local:10000/default;password=hive;user=hive
Connected to: Apache Hive (version 2.3.9)
Driver: Kyuubi Project Hive JDBC Client (version 1.9.0-SNAPSHOT)
Beeline version 1.9.0-SNAPSHOT by Apache Kyuubi
0: jdbc:hive2://hadoop-master1.orb.local:1000> select version();
+--------------------------------------------------+
|                       _c0                        |
+--------------------------------------------------+
| 2.3.9 r92dd0159f440ca7863be3232f3a683a510a62b9d  |
+--------------------------------------------------+
1 row selected (1.443 seconds)
0: jdbc:hive2://hadoop-master1.orb.local:1000>
```

---

# Checklist 📝

- [x] This patch was not authored or co-authored using [Generative Tooling](https://www.apache.org/legal/generative-tooling.html)

**Be nice. Be informative.**

Closes #6162 from pan3793/beeline-3.

Closes #6162

e85f096 [Cheng Pan] relocate
b2ce8ef [Cheng Pan] Cut out hive-common deps in beeline module

Authored-by: Cheng Pan <[email protected]>
Signed-off-by: Cheng Pan <[email protected]>
  • Loading branch information
pan3793 committed Mar 12, 2024
1 parent a5ae9b1 commit ddf347f
Show file tree
Hide file tree
Showing 24 changed files with 923 additions and 478 deletions.
3 changes: 0 additions & 3 deletions NOTICE-binary
Original file line number Diff line number Diff line change
Expand Up @@ -189,9 +189,6 @@ JUnit (4.12)

* License: Eclipse Public License

Hive Common
Copyright 2022 The Apache Software Foundation

Apache HttpClient
Copyright 1999-2020 The Apache Software Foundation

Expand Down
12 changes: 0 additions & 12 deletions kyuubi-hive-beeline/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -46,18 +46,6 @@
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-common</artifactId>
<version>${hive.version}</version>
<exclusions>
<exclusion>
<groupId>*</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client-api</artifactId>
Expand Down
141 changes: 9 additions & 132 deletions kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/BeeLine.java
Original file line number Diff line number Diff line change
Expand Up @@ -88,17 +88,14 @@
import org.apache.commons.cli.ParseException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hive.beeline.cli.CliOptionsProcessor;
import org.apache.hive.beeline.common.util.ShutdownHookManager;
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.HS2ConnectionFileParser;
import org.apache.hive.beeline.hs2connection.HS2ConnectionFileUtils;
import org.apache.hive.beeline.hs2connection.HiveSiteHS2ConnectionFileParser;
import org.apache.hive.beeline.hs2connection.UserHS2ConnectionFileParser;
import org.apache.hive.common.util.ShutdownHookManager;
import org.apache.kyuubi.jdbc.hive.JdbcConnectionParams;
import org.apache.kyuubi.jdbc.hive.JdbcUriParseException;
import org.apache.kyuubi.jdbc.hive.Utils;
Expand Down Expand Up @@ -684,66 +681,6 @@ private boolean isBeeLineOpt(String arg) {
}
}

int initArgsFromCliVars(String[] args) {
List<String> commands = Collections.emptyList();

CliOptionsProcessor optionsProcessor = new CliOptionsProcessor();
if (!optionsProcessor.process(args)) {
return 1;
}
CommandLine commandLine = optionsProcessor.getCommandLine();

Properties confProps = commandLine.getOptionProperties("hiveconf");
for (String propKey : confProps.stringPropertyNames()) {
setHiveConfVar(propKey, confProps.getProperty(propKey));
}

Properties hiveVars = commandLine.getOptionProperties("define");
for (String propKey : hiveVars.stringPropertyNames()) {
getOpts().getHiveConfVariables().put(propKey, hiveVars.getProperty(propKey));
}

Properties hiveVars2 = commandLine.getOptionProperties("hivevar");
for (String propKey : hiveVars2.stringPropertyNames()) {
getOpts().getHiveConfVariables().put(propKey, hiveVars2.getProperty(propKey));
}

getOpts().setScriptFile(commandLine.getOptionValue("f"));

if (commandLine.getOptionValues("i") != null) {
getOpts().setInitFiles(commandLine.getOptionValues("i"));
}

dbName = commandLine.getOptionValue("database");
getOpts().setVerbose(Boolean.parseBoolean(commandLine.getOptionValue("verbose")));
getOpts().setSilent(Boolean.parseBoolean(commandLine.getOptionValue("silent")));

int code = 0;
if (commandLine.getOptionValues("e") != null) {
commands = Arrays.asList(commandLine.getOptionValues("e"));
}

if (!commands.isEmpty() && getOpts().getScriptFile() != null) {
System.err.println("The '-e' and '-f' options cannot be specified simultaneously");
optionsProcessor.printCliUsage();
return 1;
}

if (!commands.isEmpty()) {
embeddedConnect();
connectDBInEmbededMode();
for (Iterator<String> i = commands.iterator(); i.hasNext(); ) {
String command = i.next().toString();
debug(loc("executing-command", command));
if (!dispatch(command)) {
code++;
}
}
exit = true; // execute and exit
}
return code;
}

int initArgs(String[] args) {
List<String> commands = Collections.emptyList();

Expand Down Expand Up @@ -884,9 +821,6 @@ private boolean connectUsingArgs(BeelineParser beelineParser, CommandLine cl) {

private void setHiveConfVar(String key, String val) {
getOpts().getHiveConfVariables().put(key, val);
if (HiveConf.ConfVars.HIVE_EXECUTION_ENGINE.varname.equals(key) && "mr".equals(val)) {
info(HiveConf.generateMrDeprecationWarning());
}
}

private String constructCmd(
Expand Down Expand Up @@ -976,15 +910,6 @@ private String obtainPasswordFromFile(String passwordFilePath) {
}
}

public void updateOptsForCli() {
getOpts().updateBeeLineOptsFromConf();
getOpts().setShowHeader(false);
getOpts().setEscapeCRLF(false);
getOpts().setOutputFormat("dsv");
getOpts().setDelimiterForDSV(' ');
getOpts().setNullEmptyString(true);
}

/**
* Start accepting input from stdin, and dispatch it to the appropriate {@link CommandHandler}
* until the global variable <code>exit</code> is true.
Expand All @@ -1005,17 +930,9 @@ public int begin(String[] args, InputStream inputStream) throws IOException {
// 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()) {
Expand Down Expand Up @@ -1194,33 +1111,10 @@ private int connectDBInEmbededMode() {
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);
Expand Down Expand Up @@ -1385,13 +1279,9 @@ boolean dispatch(String line) {
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());
}
Expand Down Expand Up @@ -1664,20 +1554,7 @@ void showWarnings(SQLWarning warn) {
}

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();
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,6 @@
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;
Expand Down Expand Up @@ -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;

Expand Down Expand Up @@ -268,20 +265,6 @@ 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()) {
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();
Expand Down Expand Up @@ -469,29 +452,15 @@ public void setShowHeader(boolean showHeader) {
}

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) {
this.escapeCRLF = escapeCRLF;
}

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) {
Expand All @@ -505,12 +474,7 @@ public void setShowDbInPrompt(boolean showDbInPrompt) {
* @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) {
Expand Down Expand Up @@ -651,11 +615,6 @@ public void setDelimiterForDSV(char delimiterForDSV) {
this.delimiterForDSV = delimiterForDSV;
}

@Ignore
public HiveConf getConf() {
return conf;
}

public void setHelpAsked(boolean helpAsked) {
this.helpAsked = helpAsked;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,20 +30,6 @@ public static ClientCommandHookFactory get() {
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) {
Expand Down Expand Up @@ -112,36 +98,20 @@ public void postHook(BeeLine beeLine) {
}

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;
}
}
}
Loading

0 comments on commit ddf347f

Please sign in to comment.