[KYUUBI #6162] Cut out hive-common deps in beeline module

# 🔍 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

e85f096b2 [Cheng Pan] relocate
b2ce8ef61 [Cheng Pan] Cut out hive-common deps in beeline module

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
This commit is contained in:
Cheng Pan 2024-03-12 11:23:10 +08:00
parent a5ae9b1a0c
commit ddf347ff77
No known key found for this signature in database
GPG Key ID: 8001952629BCC75D
24 changed files with 923 additions and 478 deletions

View File

@ -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

View File

@ -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>

View File

@ -88,9 +88,7 @@ import org.apache.commons.cli.Options;
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;
@ -98,7 +96,6 @@ 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;
@ -684,66 +681,6 @@ public class BeeLine implements Closeable {
}
}
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();
@ -884,9 +821,6 @@ public class BeeLine implements Closeable {
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(
@ -976,15 +910,6 @@ public class BeeLine implements Closeable {
}
}
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.
@ -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();
}
/**

View File

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

View File

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

View File

@ -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 Map<String, String> getHiveVariable() {
return getHiveVariables();
}
})
.substitute(conf, line);
}
return line;
}
public boolean sh(String line) {
if (line == null || line.length() == 0) {
return false;
@ -1148,7 +1088,6 @@ public class Commands {
}
line = line.substring("sh".length()).trim();
if (!beeLine.isBeeLine()) line = substituteVariables(getHiveConf(false), line.trim());
try {
ShellCmdExecutor executor =
@ -1641,9 +1580,6 @@ public class Commands {
.setConnection(new DatabaseConnection(beeLine, driver, url, props));
beeLine.getDatabaseConnection().getConnection();
if (!beeLine.isBeeLine()) {
beeLine.updateOptsForCli();
}
beeLine.runInit();
beeLine.setCompletions();

View File

@ -24,7 +24,7 @@ import java.util.*;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.hive.common.util.HiveStringUtils;
import org.apache.hive.beeline.common.util.HiveStringUtils;
import org.apache.kyuubi.util.reflect.DynConstructors;
import org.apache.kyuubi.util.reflect.DynFields;
import org.apache.kyuubi.util.reflect.DynMethods;

View File

@ -24,8 +24,8 @@ import java.io.*;
import java.nio.file.Files;
import java.sql.*;
import java.util.*;
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.KyuubiStatement;
import org.apache.kyuubi.jdbc.hive.Utils;
import org.apache.kyuubi.jdbc.hive.logs.InPlaceUpdateStream;
@ -70,7 +70,6 @@ public class KyuubiCommands extends 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;
@ -488,10 +487,6 @@ public class KyuubiCommands extends Commands {
.setConnection(new KyuubiDatabaseConnection(beeLine, driver, url, props));
beeLine.getDatabaseConnection().getConnection();
if (!beeLine.isBeeLine()) {
beeLine.updateOptsForCli();
}
// see HIVE-19048 : Initscript errors are ignored
int initScriptExecutionResult = beeLine.runInit();

View File

@ -29,7 +29,7 @@ import java.text.DecimalFormat;
import java.text.NumberFormat;
import java.util.Arrays;
import java.util.Iterator;
import org.apache.hadoop.hive.common.cli.EscapeCRLFHelper;
import org.apache.hive.beeline.common.cli.EscapeCRLFHelper;
/**
* Abstract base class representing a set of rows to be displayed. Holds column values as strings

View File

@ -1,130 +0,0 @@
/*
* 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
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* 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.cli;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.GnuParser;
import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.OptionBuilder;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
/** This class is used for parsing the options of Hive Cli */
public class CliOptionsProcessor {
private final Options options = new Options();
private org.apache.commons.cli.CommandLine commandLine;
public CliOptionsProcessor() {
// -database database
options.addOption(
OptionBuilder.hasArg()
.withArgName("databasename")
.withLongOpt("database")
.withDescription("Specify the database to use")
.create());
// -e 'quoted-query-string'
options.addOption(
OptionBuilder.hasArg()
.withArgName("quoted-query-string")
.withDescription("SQL from command line")
.create('e'));
// -f <query-file>
options.addOption(
OptionBuilder.hasArg()
.withArgName("filename")
.withDescription("SQL from " + "files")
.create('f'));
// -i <init-query-file>
options.addOption(
OptionBuilder.hasArg()
.withArgName("filename")
.withDescription("Initialization SQL file")
.create('i'));
// -hiveconf x=y
options.addOption(
OptionBuilder.withValueSeparator()
.hasArgs(2)
.withArgName("property=value")
.withLongOpt("hiveconf")
.withDescription("Use value for given property")
.create());
// Substitution option -d, --define
options.addOption(
OptionBuilder.withValueSeparator()
.hasArgs(2)
.withArgName("key=value")
.withLongOpt("define")
.withDescription(
"Variable substitution to apply to Hive commands. e" + ".g. -d A=B or --define A=B")
.create('d'));
// Substitution option --hivevar
options.addOption(
OptionBuilder.withValueSeparator()
.hasArgs(2)
.withArgName("key=value")
.withLongOpt("hivevar")
.withDescription(
"Variable substitution to apply to Hive commands. " + "e.g. --hivevar A=B")
.create());
// [-S|--silent]
options.addOption(new Option("S", "silent", false, "Silent mode in interactive shell"));
// [-v|--verbose]
options.addOption(
new Option("v", "verbose", false, "Verbose mode (echo executed SQL to the " + "console)"));
// [-H|--help]
options.addOption(new Option("H", "help", false, "Print help information"));
}
public boolean process(String[] argv) {
try {
commandLine = new GnuParser().parse(options, argv);
if (commandLine.hasOption("help")) {
printCliUsage();
return false;
}
} catch (ParseException e) {
System.err.println(e.getMessage());
printCliUsage();
return false;
}
return true;
}
public void printCliUsage() {
new HelpFormatter().printHelp("hive", options);
}
public CommandLine getCommandLine() {
return commandLine;
}
public void setCommandLine(CommandLine commandLine) {
this.commandLine = commandLine;
}
}

View File

@ -0,0 +1,78 @@
/*
* 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.common.cli;
public class EscapeCRLFHelper {
private static final char CARRIAGE_RETURN = '\r';
private static final char LINE_FEED = '\n';
public EscapeCRLFHelper() {}
/*
* Substitute for any carriage return or line feed characters in line with the escaped
* 2-character sequences \r or \n.
*
* @param line the string for the CRLF substitution.
* @return If there were no replacements, then just return line. Otherwise, a new String with
* escaped CRLF.
*/
public static String escapeCRLF(String line) {
StringBuilder sb = null;
int lastNonCRLFIndex = 0;
int index = 0;
final int length = line.length();
while (index < length) {
char ch = line.charAt(index);
if (ch == CARRIAGE_RETURN || ch == LINE_FEED) {
if (sb == null) {
// We defer allocation until we really need it since in the common case there is
// no CRLF substitution.
sb = new StringBuilder();
}
if (lastNonCRLFIndex < index) {
// Copy an intervening non-CRLF characters up to but not including current 'index'.
sb.append(line.substring(lastNonCRLFIndex, index));
}
lastNonCRLFIndex = ++index;
if (ch == CARRIAGE_RETURN) {
sb.append("\\r");
} else {
sb.append("\\n");
}
} else {
index++;
}
}
if (sb == null) {
// No CRLF substitution -- return original line.
return line;
} else {
if (lastNonCRLFIndex < index) {
// Copy an intervening non-CRLF characters up to but not including current 'index'.
sb.append(line.substring(lastNonCRLFIndex, index));
}
return sb.toString();
}
}
}

View File

@ -0,0 +1,53 @@
/*
* 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
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* 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.common.cli;
import java.io.IOException;
import java.io.PrintStream;
import org.apache.hive.beeline.common.util.StreamPrinter;
public class ShellCmdExecutor {
private String cmd;
private PrintStream out;
private PrintStream err;
public ShellCmdExecutor(String cmd, PrintStream out, PrintStream err) {
this.cmd = cmd;
this.out = out;
this.err = err;
}
public int execute() throws Exception {
try {
Process executor = Runtime.getRuntime().exec(cmd);
StreamPrinter outPrinter = new StreamPrinter(executor.getInputStream(), null, out);
StreamPrinter errPrinter = new StreamPrinter(executor.getErrorStream(), null, err);
outPrinter.start();
errPrinter.start();
int ret = executor.waitFor();
outPrinter.join();
errPrinter.join();
return ret;
} catch (IOException ex) {
throw new Exception("Failed to execute " + cmd, ex);
}
}
}

View File

@ -0,0 +1,190 @@
/*
* 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
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* 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.common.log;
import static org.fusesource.jansi.Ansi.ansi;
import com.google.common.base.Function;
import com.google.common.collect.Lists;
import java.io.PrintStream;
import java.io.StringWriter;
import java.text.DecimalFormat;
import java.util.List;
import javax.annotation.Nullable;
import org.apache.commons.lang3.StringUtils;
import org.fusesource.jansi.Ansi;
/** Renders information from ProgressMonitor to the stream provided. */
public class InPlaceUpdate {
public static final int MIN_TERMINAL_WIDTH = 94;
// keep this within 80 chars width. If more columns needs to be added then update min terminal
// width requirement and SEPARATOR width accordingly
private static final String HEADER_FORMAT = "%16s%10s %13s %5s %9s %7s %7s %6s %6s ";
private static final String VERTEX_FORMAT = "%-16s%10s %13s %5s %9s %7s %7s %6s %6s ";
private static final String FOOTER_FORMAT = "%-15s %-30s %-4s %-25s";
private static final int PROGRESS_BAR_CHARS = 30;
private static final String SEPARATOR =
new String(new char[MIN_TERMINAL_WIDTH]).replace("\0", "-");
/* Pretty print the values */
private final DecimalFormat secondsFormatter = new DecimalFormat("#0.00");
private int lines = 0;
private PrintStream out;
public InPlaceUpdate(PrintStream out) {
this.out = out;
}
public InPlaceUpdate() {
this(System.out);
}
public static void reprintLine(PrintStream out, String line) {
out.print(ansi().eraseLine(Ansi.Erase.ALL).a(line).a('\n').toString());
out.flush();
}
public static void rePositionCursor(PrintStream ps) {
ps.print(ansi().cursorUp(0).toString());
ps.flush();
}
/**
* NOTE: Use this method only if isUnixTerminal is true. Erases the current line and prints the
* given line.
*
* @param line - line to print
*/
private void reprintLine(String line) {
reprintLine(out, line);
lines++;
}
/**
* NOTE: Use this method only if isUnixTerminal is true. Erases the current line and prints the
* given line with the specified color.
*
* @param line - line to print
* @param color - color for the line
*/
private void reprintLineWithColorAsBold(String line, Ansi.Color color) {
out.print(
ansi()
.eraseLine(Ansi.Erase.ALL)
.fg(color)
.bold()
.a(line)
.a('\n')
.boldOff()
.reset()
.toString());
out.flush();
lines++;
}
/**
* NOTE: Use this method only if isUnixTerminal is true. Erases the current line and prints the
* given multiline. Make sure the specified line is not terminated by linebreak.
*
* @param line - line to print
*/
private void reprintMultiLine(String line) {
int numLines = line.split("\r\n|\r|\n").length;
out.print(ansi().eraseLine(Ansi.Erase.ALL).a(line).a('\n').toString());
out.flush();
lines += numLines;
}
/**
* NOTE: Use this method only if isUnixTerminal is true. Repositions the cursor back to line 0.
*/
private void repositionCursor() {
if (lines > 0) {
out.print(ansi().cursorUp(lines).toString());
out.flush();
lines = 0;
}
}
// [==================>>-----]
private String getInPlaceProgressBar(double percent) {
StringWriter bar = new StringWriter();
bar.append("[");
int remainingChars = PROGRESS_BAR_CHARS - 4;
int completed = (int) (remainingChars * percent);
int pending = remainingChars - completed;
for (int i = 0; i < completed; i++) {
bar.append("=");
}
bar.append(">>");
for (int i = 0; i < pending; i++) {
bar.append("-");
}
bar.append("]");
return bar.toString();
}
public void render(ProgressMonitor monitor) {
if (monitor == null) return;
// position the cursor to line 0
repositionCursor();
// print header
// -------------------------------------------------------------------------------
// VERTICES STATUS TOTAL COMPLETED RUNNING PENDING FAILED KILLED
// -------------------------------------------------------------------------------
reprintLine(SEPARATOR);
reprintLineWithColorAsBold(
String.format(HEADER_FORMAT, monitor.headers().toArray()), Ansi.Color.CYAN);
reprintLine(SEPARATOR);
// Map 1 .......... container SUCCEEDED 7 7 0 0 0 0
List<String> printReady =
Lists.transform(
monitor.rows(),
new Function<List<String>, String>() {
@Nullable
@Override
public String apply(@Nullable List<String> row) {
return String.format(VERTEX_FORMAT, row.toArray());
}
});
reprintMultiLine(StringUtils.join(printReady, "\n"));
// -------------------------------------------------------------------------------
// VERTICES: 03/04 [=================>>-----] 86% ELAPSED TIME: 1.71 s
// -------------------------------------------------------------------------------
String progressStr = "" + (int) (monitor.progressedPercentage() * 100) + "%";
float et = (float) (System.currentTimeMillis() - monitor.startTime()) / (float) 1000;
String elapsedTime = "ELAPSED TIME: " + secondsFormatter.format(et) + " s";
String footer =
String.format(
FOOTER_FORMAT,
monitor.footerSummary(),
getInPlaceProgressBar(monitor.progressedPercentage()),
progressStr,
elapsedTime);
reprintLine(SEPARATOR);
reprintLineWithColorAsBold(footer, Ansi.Color.RED);
reprintLine(SEPARATOR);
}
}

View File

@ -0,0 +1,69 @@
/*
* 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
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* 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.common.log;
import java.util.Collections;
import java.util.List;
public interface ProgressMonitor {
ProgressMonitor NULL =
new ProgressMonitor() {
@Override
public List<String> headers() {
return Collections.emptyList();
}
@Override
public List<List<String>> rows() {
return Collections.emptyList();
}
@Override
public String footerSummary() {
return "";
}
@Override
public long startTime() {
return 0;
}
@Override
public String executionStatus() {
return "";
}
@Override
public double progressedPercentage() {
return 0;
}
};
List<String> headers();
List<List<String>> rows();
String footerSummary();
long startTime();
String executionStatus();
double progressedPercentage();
}

View File

@ -0,0 +1,113 @@
/*
* 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
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* 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.common.util;
import com.google.common.base.Splitter;
import java.util.Iterator;
/**
* HiveStringUtils General string utils
*
* <p>Originally 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;
}
Iterator<String> iterator = Splitter.on("\n").omitEmptyStrings().split(statement).iterator();
int[] startQuote = {-1};
StringBuilder ret = new StringBuilder(statement.length());
while (iterator.hasNext()) {
String lineWithComments = iterator.next();
String lineNoComments = removeComments(lineWithComments, startQuote);
ret.append(lineNoComments);
if (iterator.hasNext() && !lineNoComments.isEmpty()) {
ret.append("\n");
}
}
return ret.toString();
}
/**
* Remove comments from the current line of a query. Avoid removing comment-like strings inside
* quotes.
*
* @param line a line of sql text
* @param startQuote The value -1 indicates that line does not begin inside a string literal.
* Other values indicate that line does begin inside a string literal and the value passed is
* the delimiter character. The array type is used to pass int type as input/output parameter.
* @return the line with comments removed.
*/
public static String removeComments(String line, int[] startQuote) {
if (line == null || line.isEmpty()) {
return line;
}
if (startQuote[0] == -1 && isComment(line)) {
return ""; // assume # can only be used at the beginning of line.
}
StringBuilder builder = new StringBuilder();
for (int index = 0; index < line.length(); ) {
if (startQuote[0] == -1
&& index < line.length() - 1
&& line.charAt(index) == '-'
&& line.charAt(index + 1) == '-') {
// Jump to the end of current line. When a multiple line query is executed with -e
// parameter,
// it is passed in as one line string separated with '\n'
for (; index < line.length() && line.charAt(index) != '\n'; ++index) ;
continue;
}
char letter = line.charAt(index);
if (startQuote[0] == letter && (index == 0 || line.charAt(index - 1) != '\\')) {
startQuote[0] = -1; // Turn escape off.
} else if (startQuote[0] == -1
&& (letter == '\'' || letter == '"')
&& (index == 0 || line.charAt(index - 1) != '\\')) {
startQuote[0] = letter; // Turn escape on.
}
builder.append(letter);
index++;
}
return builder.toString().trim();
}
/**
* Test whether a line is a comment.
*
* @param line the line to be tested
* @return true if a comment
*/
private static boolean isComment(String line) {
// SQL92 comment prefix is "--"
// beeline also supports shell-style "#" prefix
String lineTrimmed = line.trim();
return lineTrimmed.startsWith("#") || lineTrimmed.startsWith("--");
}
}

View File

@ -0,0 +1,125 @@
/*
* 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
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* 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.common.util;
import com.google.common.annotations.VisibleForTesting;
import java.io.File;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import org.apache.hadoop.fs.FileSystem;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This is just a wrapper around hadoop's ShutdownHookManager but also manages delete on exit hook
* for temp files.
*/
public class ShutdownHookManager {
private static final org.apache.hadoop.util.ShutdownHookManager MGR =
org.apache.hadoop.util.ShutdownHookManager.get();
private static final DeleteOnExitHook DELETE_ON_EXIT_HOOK = new DeleteOnExitHook();
private static final Logger LOG = LoggerFactory.getLogger(ShutdownHookManager.class.getName());
static {
MGR.addShutdownHook(DELETE_ON_EXIT_HOOK, -1);
}
/**
* Adds shutdown hook with default priority (10)
*
* @param shutdownHook - shutdown hook
*/
public static void addShutdownHook(Runnable shutdownHook) {
addShutdownHook(shutdownHook, FileSystem.SHUTDOWN_HOOK_PRIORITY);
}
/**
* Adds a shutdownHook with a priority, the higher the priority the earlier will run.
* ShutdownHooks with same priority run in a non-deterministic order.
*
* @param shutdownHook shutdownHook <code>Runnable</code>
* @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<File> deleteTargets = Collections.synchronizedSet(new HashSet<File>());
@Override
public void run() {
for (File deleteTarget : deleteTargets) {
deleteTarget.delete();
}
deleteTargets.clear();
}
}
}

View File

@ -0,0 +1,64 @@
/*
* 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
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* 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.common.util;
import java.io.*;
import org.apache.hadoop.io.IOUtils;
/** StreamPrinter. */
public class StreamPrinter extends Thread {
InputStream is;
String type;
PrintStream[] outputStreams;
public StreamPrinter(InputStream is, String type, PrintStream... outputStreams) {
this.is = is;
this.type = type;
this.outputStreams = outputStreams;
}
@Override
public void run() {
BufferedReader br = null;
try {
InputStreamReader isr = new InputStreamReader(is);
br = new BufferedReader(isr);
String line = null;
if (type != null) {
while ((line = br.readLine()) != null) {
for (PrintStream os : outputStreams) {
os.println(type + ">" + line);
}
}
} else {
while ((line = br.readLine()) != null) {
for (PrintStream os : outputStreams) {
os.println(line);
}
}
}
br.close();
br = null;
} catch (IOException ioe) {
ioe.printStackTrace();
} finally {
IOUtils.closeStream(br);
}
}
}

View File

@ -18,14 +18,13 @@
package org.apache.hive.beeline.hs2connection;
import com.google.common.annotations.VisibleForTesting;
import java.io.File;
import java.net.InetAddress;
import java.net.URI;
import java.net.URL;
import java.net.UnknownHostException;
import java.util.Properties;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.common.ServerUtils;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -33,27 +32,84 @@ import org.slf4j.LoggerFactory;
* Looks for a hive-site.xml from the classpath. If found this class parses the hive-site.xml
* to return a set of connection properties which can be used to construct the connection url
* for Beeline connection
*
* This class is modified to get rid of dependency on HiveConf
*/
public class HiveSiteHS2ConnectionFileParser implements HS2ConnectionFileParser {
private Configuration conf;
private final URL hiveSiteURI;
private static final String TRUSTSTORE_PASS_PROP = "javax.net.ssl.trustStorePassword";
private static final String TRUSTSTORE_PROP = "javax.net.ssl.trustStore";
private static final Logger log = LoggerFactory.getLogger(HiveSiteHS2ConnectionFileParser.class);
private static final Logger LOG = LoggerFactory.getLogger(HiveSiteHS2ConnectionFileParser.class);
// Copied from HiveConf
private static URL findConfigFile(ClassLoader classLoader, String name, boolean doLog) {
URL result = classLoader.getResource(name);
if (result == null) {
String confPath = System.getenv("HIVE_CONF_DIR");
result = checkConfigFile(new File(confPath, name));
if (result == null) {
String homePath = System.getenv("HIVE_HOME");
String nameInConf = "conf" + File.separator + name;
result = checkConfigFile(new File(homePath, nameInConf));
if (result == null) {
URI jarUri = null;
try {
// Handle both file:// and jar:<url>!{entry} in the case of shaded hive libs
URL sourceUrl =
HiveSiteHS2ConnectionFileParser.class
.getProtectionDomain()
.getCodeSource()
.getLocation();
jarUri =
sourceUrl.getProtocol().equalsIgnoreCase("jar")
? new URI(sourceUrl.getPath())
: sourceUrl.toURI();
} catch (Throwable e) {
LOG.info("Cannot get jar URI", e);
}
// From the jar file, the parent is /lib folder
File parent = new File(jarUri).getParentFile();
if (parent != null) {
result = checkConfigFile(new File(parent.getParentFile(), nameInConf));
}
}
}
}
if (doLog) {
LOG.info("Found configuration file {}", result);
}
return result;
}
// Copied from HiveConf
private static URL checkConfigFile(File f) {
try {
return (f.exists() && f.isFile()) ? f.toURI().toURL() : null;
} catch (Throwable e) {
LOG.info("Error looking for config {}", f, e);
return null;
}
}
public HiveSiteHS2ConnectionFileParser() {
hiveSiteURI = HiveConf.getHiveSiteLocation();
ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
if (classLoader == null) {
classLoader = this.getClass().getClassLoader();
}
// Look for hive-site.xml on the CLASSPATH and log its location if found.
hiveSiteURI = findConfigFile(classLoader, "hive-site.xml", true);
conf = new Configuration();
if (hiveSiteURI == null) {
log.debug("hive-site.xml not found for constructing the connection URL");
LOG.debug("hive-site.xml not found for constructing the connection URL");
} else {
log.info("Using hive-site.xml at " + hiveSiteURI);
LOG.info("Using hive-site.xml at " + hiveSiteURI);
conf.addResource(hiveSiteURI);
}
}
@VisibleForTesting
void setHiveConf(HiveConf hiveConf) {
void setHiveConf(Configuration hiveConf) {
this.conf = hiveConf;
}
@ -72,7 +128,7 @@ public class HiveSiteHS2ConnectionFileParser implements HS2ConnectionFileParser
}
private void addSSL(Properties props) {
if (!HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_SERVER2_USE_SSL)) {
if (!conf.getBoolean("hive.server2.use.SSL", false)) {
return;
} else {
props.setProperty("ssl", "true");
@ -85,32 +141,31 @@ public class HiveSiteHS2ConnectionFileParser implements HS2ConnectionFileParser
if (trustStorePassword != null && !trustStorePassword.isEmpty()) {
props.setProperty("trustStorePassword", trustStorePassword);
}
String saslQop = HiveConf.getVar(conf, ConfVars.HIVE_SERVER2_THRIFT_SASL_QOP);
String saslQop = conf.get("hive.server2.thrift.sasl.qop", "auth");
if (!"auth".equalsIgnoreCase(saslQop)) {
props.setProperty("sasl.qop", saslQop);
}
}
private void addKerberos(Properties props) {
if ("KERBEROS".equals(HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_SERVER2_AUTHENTICATION))) {
if ("KERBEROS".equals(conf.get("hive.server2.authentication", "NONE"))) {
props.setProperty(
"principal", HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL));
"principal", conf.get("hive.server2.authentication.kerberos.principal", ""));
}
}
private void addHttp(Properties props) {
if ("http".equalsIgnoreCase(HiveConf.getVar(conf, ConfVars.HIVE_SERVER2_TRANSPORT_MODE))) {
if ("http".equalsIgnoreCase(conf.get("hive.server2.transport.mode", "binary"))) {
props.setProperty("transportMode", "http");
} else {
return;
}
props.setProperty("httpPath", HiveConf.getVar(conf, ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH));
props.setProperty("httpPath", conf.get("hive.server2.thrift.http.path", "cliservice"));
}
private void addHosts(Properties props) throws BeelineHS2ConnectionFileParseException {
// if zk HA is enabled get hosts property
if (HiveConf.getBoolVar(
conf, HiveConf.ConfVars.HIVE_SERVER2_SUPPORT_DYNAMIC_SERVICE_DISCOVERY)) {
if (conf.getBoolean("hive.server2.support.dynamic.service.discovery", false)) {
addZKServiceDiscoveryHosts(props);
} else {
addDefaultHS2Hosts(props);
@ -121,25 +176,42 @@ public class HiveSiteHS2ConnectionFileParser implements HS2ConnectionFileParser
throws BeelineHS2ConnectionFileParseException {
props.setProperty("serviceDiscoveryMode", "zooKeeper");
props.setProperty(
"zooKeeperNamespace", HiveConf.getVar(conf, ConfVars.HIVE_SERVER2_ZOOKEEPER_NAMESPACE));
props.setProperty("hosts", HiveConf.getVar(conf, ConfVars.HIVE_ZOOKEEPER_QUORUM));
"zooKeeperNamespace", conf.get("hive.server2.zookeeper.namespace", "hiveserver2"));
props.setProperty("hosts", conf.get("hive.zookeeper.quorum", ""));
}
/**
* Get the Inet address of the machine of the given host name.
*
* @param hostname The name of the host
* @return The network address of the the host
* @throws UnknownHostException
*/
// Copied from ServerUtils
private static InetAddress getHostAddress(String hostname) throws UnknownHostException {
InetAddress serverIPAddress;
if (hostname != null && !hostname.isEmpty()) {
serverIPAddress = InetAddress.getByName(hostname);
} else {
serverIPAddress = InetAddress.getLocalHost();
}
return serverIPAddress;
}
private void addDefaultHS2Hosts(Properties props) throws BeelineHS2ConnectionFileParseException {
String hiveHost = System.getenv("HIVE_SERVER2_THRIFT_BIND_HOST");
if (hiveHost == null) {
hiveHost = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST);
hiveHost = conf.get("hive.server2.thrift.bind.host", "");
}
InetAddress serverIPAddress;
try {
serverIPAddress = ServerUtils.getHostAddress(hiveHost);
serverIPAddress = getHostAddress(hiveHost);
} catch (UnknownHostException e) {
throw new BeelineHS2ConnectionFileParseException(e.getMessage(), e);
}
int portNum =
getPortNum(
"http".equalsIgnoreCase(HiveConf.getVar(conf, ConfVars.HIVE_SERVER2_TRANSPORT_MODE)));
getPortNum("http".equalsIgnoreCase(conf.get("hive.server2.transport.mode", "binary")));
props.setProperty("hosts", serverIPAddress.getHostName() + ":" + portNum);
}
@ -151,14 +223,14 @@ public class HiveSiteHS2ConnectionFileParser implements HS2ConnectionFileParser
if (portString != null) {
portNum = Integer.parseInt(portString);
} else {
portNum = HiveConf.getIntVar(conf, ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT);
portNum = conf.getInt("hive.server2.thrift.http.port", 10001);
}
} else {
portString = System.getenv("HIVE_SERVER2_THRIFT_PORT");
if (portString != null) {
portNum = Integer.parseInt(portString);
} else {
portNum = HiveConf.getIntVar(conf, ConfVars.HIVE_SERVER2_THRIFT_PORT);
portNum = conf.getInt("hive.server2.thrift.port", 10000);
}
}
return portNum;

View File

@ -19,8 +19,8 @@ package org.apache.hive.beeline.logs;
import java.io.PrintStream;
import java.util.List;
import org.apache.hadoop.hive.common.log.InPlaceUpdate;
import org.apache.hadoop.hive.common.log.ProgressMonitor;
import org.apache.hive.beeline.common.log.InPlaceUpdate;
import org.apache.hive.beeline.common.log.ProgressMonitor;
import org.apache.kyuubi.jdbc.hive.logs.InPlaceUpdateStream;
import org.apache.kyuubi.shaded.hive.service.rpc.thrift.TJobExecutionStatus;
import org.apache.kyuubi.shaded.hive.service.rpc.thrift.TProgressUpdateResp;

View File

@ -33,7 +33,7 @@ import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.hive.common.util.HiveTestUtils;
import org.apache.hive.beeline.common.HiveTestUtils;
import org.apache.kyuubi.util.JavaUtils;
import org.junit.Assert;
import org.junit.Test;

View File

@ -47,21 +47,6 @@ public class TestClientCommandHookFactory {
return mockBeeLine;
}
@Test
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);

View File

@ -18,7 +18,7 @@
package org.apache.hive.beeline;
import static org.apache.hive.common.util.HiveStringUtils.removeComments;
import static org.apache.hive.beeline.common.util.HiveStringUtils.removeComments;
import static org.junit.Assert.assertEquals;
import java.io.IOException;

View File

@ -0,0 +1,104 @@
/*
* 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
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* 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.common;
import com.google.common.io.Files;
import java.io.*;
import java.net.URL;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.zip.ZipEntry;
import java.util.zip.ZipOutputStream;
import org.apache.commons.io.IOUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class HiveTestUtils {
public static final Logger LOG = LoggerFactory.getLogger(HiveTestUtils.class);
public static final String JAVA_FILE_EXT = ".java";
public static final String CLAZZ_FILE_EXT = ".class";
public static final String JAR_FILE_EXT = ".jar";
public static final String TXT_FILE_EXT = ".txt";
public static String getFileFromClasspath(String name) {
URL url = ClassLoader.getSystemResource(name);
if (url == null) {
throw new IllegalArgumentException("Could not find " + name);
}
return url.getPath();
}
private static void executeCmd(String[] cmdArr, File dir)
throws IOException, InterruptedException {
final Process p1 = Runtime.getRuntime().exec(cmdArr, null, dir);
new Thread(
new Runnable() {
@Override
public void run() {
BufferedReader input =
new BufferedReader(new InputStreamReader(p1.getErrorStream()));
String line;
try {
while ((line = input.readLine()) != null) {
System.out.println(line);
}
} catch (IOException e) {
LOG.error("Failed to execute the command due the exception " + e);
}
}
})
.start();
p1.waitFor();
}
public static File genLocalJarForTest(String pathToClazzFile, String clazzName)
throws IOException, InterruptedException {
return genLocalJarForTest(pathToClazzFile, clazzName, new HashMap<File, String>());
}
public static File genLocalJarForTest(
String pathToClazzFile, String clazzName, Map<File, String> extraContent)
throws IOException, InterruptedException {
String u = pathToClazzFile;
File dir = new File(u);
File parentDir = dir.getParentFile();
File f = new File(parentDir, clazzName + JAVA_FILE_EXT);
Files.copy(dir, f);
executeCmd(new String[] {"javac", clazzName + JAVA_FILE_EXT}, parentDir);
f.delete();
File outputJar = new File(parentDir, clazzName + JAR_FILE_EXT);
ZipOutputStream zos = new ZipOutputStream(new FileOutputStream(outputJar));
String contentClassName = clazzName + CLAZZ_FILE_EXT;
zos.putNextEntry(new ZipEntry(contentClassName));
IOUtils.copy(new FileInputStream(new File(parentDir, contentClassName)), zos);
zos.closeEntry();
for (Entry<File, String> entry : extraContent.entrySet()) {
zos.putNextEntry(new ZipEntry(entry.getKey().toString()));
zos.write(entry.getValue().getBytes());
zos.closeEntry();
}
zos.close();
new File(parentDir, contentClassName).delete();
return outputJar;
}
}

View File

@ -20,7 +20,7 @@ package org.apache.hive.beeline.hs2connection;
import java.io.File;
import java.util.ArrayList;
import java.util.List;
import org.apache.hive.common.util.HiveTestUtils;
import org.apache.hive.beeline.common.HiveTestUtils;
import org.junit.After;
import org.junit.Assert;
import org.junit.Test;