Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 6 additions & 1 deletion beeline/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@
<artifactId>jackson-core</artifactId>
</dependency>
<dependency>
<groupId>jline</groupId>
<groupId>org.jline</groupId>
<artifactId>jline</artifactId>
</dependency>
<dependency>
Expand All @@ -107,6 +107,11 @@
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.thrift</groupId>
<artifactId>libthrift</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,8 @@
import java.util.LinkedList;
import java.util.List;

import jline.console.completer.Completer;
import jline.console.completer.NullCompleter;
import org.jline.reader.Completer;
import org.jline.reader.impl.completer.NullCompleter;

/**
* An abstract implementation of CommandHandler.
Expand Down
128 changes: 70 additions & 58 deletions beeline/src/java/org/apache/hive/beeline/BeeLine.java
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import java.io.EOFException;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOError;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
Expand Down Expand Up @@ -100,6 +101,7 @@
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.MatchingStringsCompleter;
import org.apache.hive.common.util.ShutdownHookManager;
import org.apache.hive.common.util.HiveStringUtils;
import org.apache.hive.jdbc.HiveConnection;
Expand All @@ -110,11 +112,16 @@

import com.google.common.annotations.VisibleForTesting;

import jline.console.ConsoleReader;
import jline.console.completer.Completer;
import jline.console.completer.FileNameCompleter;
import jline.console.completer.StringsCompleter;
import jline.console.history.FileHistory;
import org.jline.reader.Completer;
import org.jline.reader.History;
import org.jline.reader.LineReader;
import org.jline.reader.LineReaderBuilder;
import org.jline.reader.impl.LineReaderImpl;
import org.jline.reader.impl.history.DefaultHistory;
import org.jline.terminal.TerminalBuilder;

import static org.jline.builtins.Completers.FileNameCompleter;


/**
* A console SQL shell with command completion.
Expand Down Expand Up @@ -152,13 +159,13 @@ public class BeeLine implements Closeable {
private PrintStream outputStream = new PrintStream(System.out, true);
private PrintStream errorStream = new PrintStream(System.err, true);
private InputStream inputStream = System.in;
private ConsoleReader consoleReader;
private LineReader lineReader;
private List<String> batch = null;
private final Reflector reflector = new Reflector(this);
private String dbName = null;
private String currentDatabase = null;

private FileHistory history;
private History history;
// Indicates if this instance of beeline is running in compatibility mode, or beeline mode
private boolean isBeeLine = true;

Expand Down Expand Up @@ -204,7 +211,7 @@ public class BeeLine implements Closeable {
new ReflectiveCommandHandler(this, new String[] {"quit", "done", "exit"},
null),
new ReflectiveCommandHandler(this, new String[] {"connect", "open"},
new Completer[] {new StringsCompleter(getConnectionURLExamples())}),
new Completer[] {new MatchingStringsCompleter(getConnectionURLExamples())}),
new ReflectiveCommandHandler(this, new String[] {"describe"},
new Completer[] {new TableNameCompletor(this)}),
new ReflectiveCommandHandler(this, new String[] {"indexes"},
Expand Down Expand Up @@ -233,7 +240,7 @@ public class BeeLine implements Closeable {
null),
new ReflectiveCommandHandler(this, new String[] {"metadata"},
new Completer[] {
new StringsCompleter(getMetadataMethodNames())}),
new MatchingStringsCompleter(getMetadataMethodNames())}),
new ReflectiveCommandHandler(this, new String[] {"nativesql"},
null),
new ReflectiveCommandHandler(this, new String[] {"dbinfo"},
Expand Down Expand Up @@ -263,9 +270,9 @@ public class BeeLine implements Closeable {
new ReflectiveCommandHandler(this, new String[] {"closeall"},
null),
new ReflectiveCommandHandler(this, new String[] {"isolation"},
new Completer[] {new StringsCompleter(getIsolationLevels())}),
new Completer[] {new MatchingStringsCompleter(getIsolationLevels())}),
new ReflectiveCommandHandler(this, new String[] {"outputformat"},
new Completer[] {new StringsCompleter(
new Completer[] {new MatchingStringsCompleter(
formats.keySet().toArray(new String[0]))}),
new ReflectiveCommandHandler(this, new String[] {"autocommit"},
null),
Expand Down Expand Up @@ -309,9 +316,9 @@ public class BeeLine implements Closeable {

static {
try {
Class.forName("jline.console.ConsoleReader");
Class.forName("org.jline.reader.LineReader");
} catch (Throwable t) {
throw new ExceptionInInitializerError("jline-missing");
throw new ExceptionInInitializerError("jline3-missing");
}
}

Expand Down Expand Up @@ -400,7 +407,7 @@ public class BeeLine implements Closeable {
.withLongOpt("help")
.withDescription("Display this message")
.create('h'));

// -getUrlsFromBeelineSite
options.addOption(OptionBuilder
.withLongOpt("getUrlsFromBeelineSite")
Expand Down Expand Up @@ -569,19 +576,15 @@ public BeeLine() {
public BeeLine(boolean isBeeLine) {
this.isBeeLine = isBeeLine;
this.signalHandler = new SunSignalHandler(this);
this.shutdownHook = new Runnable() {
@Override
public void run() {
try {
if (history != null) {
history.setMaxSize(getOpts().getMaxHistoryRows());
history.flush();
}
} catch (IOException e) {
error(e);
} finally {
close();
this.shutdownHook = () -> {
try {
if (history != null) {
history.save();
}
} catch (IOException e) {
error(e);
} finally {
close();
}
};
}
Expand Down Expand Up @@ -863,7 +866,7 @@ private boolean connectUsingArgs(BeelineParser beelineParser, CommandLine cl) {
getOpts().setHelpAsked(true);
return true;
}

if (cl.hasOption("getUrlsFromBeelineSite")) {
printBeelineSiteUrls();
getOpts().setBeelineSiteUrlsAsked(true);
Expand Down Expand Up @@ -937,8 +940,8 @@ private boolean connectUsingArgs(BeelineParser beelineParser, CommandLine cl) {
String propertyFile = cl.getOptionValue("property-file");
if (propertyFile != null) {
try {
this.consoleReader = new ConsoleReader();
} catch (IOException e) {
this.lineReader = LineReaderBuilder.builder().build();
} catch (IOError e) {
handleException(e);
}
if (!dispatch("!properties " + propertyFile)) {
Expand Down Expand Up @@ -980,7 +983,7 @@ private void printBeelineSiteUrls() {
}
}
}

private boolean isZkBasedUrl(String urlFromBeelineSite) {
String zkJdbcUriParam = ("serviceDiscoveryMode=zooKeeper").toLowerCase();
if (urlFromBeelineSite.toLowerCase().contains(zkJdbcUriParam)) {
Expand Down Expand Up @@ -1116,9 +1119,9 @@ public int begin(String[] args, InputStream inputStream, boolean keepHistory) th
//add shutdown hook to cleanup the beeline for smooth exit
addBeelineShutdownHook();

//this method also initializes the consoleReader which is
//this method also initializes the lineReader which is
//needed by initArgs for certain execution paths
ConsoleReader reader = initializeConsoleReader(inputStream);
initializeLineReader(inputStream);
if (isBeeLine) {
int code = initArgs(args);
if (code != 0) {
Expand Down Expand Up @@ -1146,7 +1149,7 @@ public int begin(String[] args, InputStream inputStream, boolean keepHistory) th
} catch (Exception e) {
// ignore
}
return execute(reader, false);
return execute(lineReader, false);
}

/*
Expand Down Expand Up @@ -1350,7 +1353,7 @@ private int executeFile(String fileName) {
}
fileStream = fs.open(path);
}
return execute(initializeConsoleReader(fileStream), !getOpts().getForce());
return execute(initializeLineReader(fileStream), !getOpts().getForce());
} catch (Throwable t) {
handleException(t);
return ERRNO_OTHER;
Expand All @@ -1359,10 +1362,10 @@ private int executeFile(String fileName) {
}
}

private int execute(ConsoleReader reader, boolean exitOnError) {
private int execute(LineReader reader, boolean exitOnError) {
int lastExecutionResult = ERRNO_OK;
Character mask = (System.getProperty("jline.terminal", "").equals("jline.UnsupportedTerminal")) ? null
: ConsoleReader.NULL_MASK;
: LineReaderImpl.NULL_MASK;

while (!exit) {
try {
Expand Down Expand Up @@ -1403,48 +1406,58 @@ private void setupHistory() throws IOException {
return;
}

this.history = new FileHistory(new File(getOpts().getHistoryFile()));
this.history = new DefaultHistory();
}

private void addBeelineShutdownHook() throws IOException {
// add shutdown hook to flush the history to history file and it also close all open connections
ShutdownHookManager.addShutdownHook(getShutdownHook());
}

public ConsoleReader initializeConsoleReader(InputStream inputStream) throws IOException {
public LineReader initializeLineReader(InputStream inputStream) throws IOException {
final LineReaderBuilder builder = LineReaderBuilder.builder();
if (inputStream != null) {
// ### NOTE: fix for sf.net bug 879425.
// Working around an issue in jline-2.1.2, see https://github.com/jline/jline/issues/10
// by appending a newline to the end of inputstream
InputStream inputStreamAppendedNewline = new SequenceInputStream(inputStream,
new ByteArrayInputStream((new String("\n")).getBytes()));
consoleReader = new ConsoleReader(inputStreamAppendedNewline, getErrorStream());
consoleReader.setCopyPasteDetection(true); // jline will detect if <tab> is regular character
builder.terminal(TerminalBuilder.builder()
.streams(inputStreamAppendedNewline, getErrorStream()).build());
} else {
consoleReader = new ConsoleReader(getInputStream(), getErrorStream());
builder.terminal(TerminalBuilder.builder()
.streams(getInputStream(), getErrorStream()).build());
}

//disable the expandEvents for the purpose of backward compatibility
consoleReader.setExpandEvents(false);
if (inputStream instanceof FileInputStream || inputStream instanceof FSDataInputStream) {
// from script.. no need to load history and no need of completer, either
lineReader = builder.build();
return lineReader;
}

try {
// now set the output for the history
if (this.history != null) {
consoleReader.setHistory(this.history);
} else {
consoleReader.setHistoryEnabled(false);
builder.history(this.history);
builder.variable(LineReader.HISTORY_FILE, new File(getOpts().getHistoryFile()));
builder.variable(LineReader.HISTORY_FILE_SIZE, getOpts().getMaxHistoryRows());
// in-memory keep more data, but at least 500 entries
builder.variable(LineReader.HISTORY_SIZE, Math.max(500, 3 * getOpts().getMaxHistoryRows()));
}
} catch (Exception e) {
handleException(e);
}

if (inputStream instanceof FileInputStream || inputStream instanceof FSDataInputStream) {
// from script.. no need to load history and no need of completer, either
return consoleReader;
builder.completer(new BeeLineCompleter(this));
lineReader = builder.build();
lineReader.unsetOpt(LineReader.Option.HISTORY_TIMESTAMPED);

if (this.history != null) {
this.history.attach(lineReader);
}

consoleReader.addCompleter(new BeeLineCompleter(this));
return consoleReader;
return lineReader;

}

void usage() {
Expand Down Expand Up @@ -1495,13 +1508,12 @@ boolean dispatch(String line) {
}

line = HiveStringUtils.removeComments(line);
line = line.trim();

if (line.trim().length() == 0) {
if (line.length() == 0) {
return true;
}

line = line.trim();

// save it to the current script, if any
if (scriptOutputFile != null) {
scriptOutputFile.addLine(line);
Expand Down Expand Up @@ -2497,12 +2509,12 @@ InputStream getInputStream() {
return inputStream;
}

ConsoleReader getConsoleReader() {
return consoleReader;
LineReader getLineReader() {
return lineReader;
}

void setConsoleReader(ConsoleReader reader) {
this.consoleReader = reader;
void setLineReader(LineReader reader) {
this.lineReader = reader;
}

List<String> getBatch() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,32 +21,32 @@
import java.util.LinkedList;
import java.util.List;

import jline.console.completer.AggregateCompleter;
import jline.console.completer.Completer;
import jline.console.completer.NullCompleter;
import jline.console.completer.StringsCompleter;
import org.apache.hive.common.util.MatchingStringsCompleter;
import org.jline.reader.Completer;
import org.jline.reader.impl.completer.AggregateCompleter;
import org.jline.reader.impl.completer.NullCompleter;

class BeeLineCommandCompleter extends AggregateCompleter {
public BeeLineCommandCompleter(Iterable<CommandHandler> handlers) {
super(getCompleters(handlers));
}

public static List<Completer> getCompleters(Iterable<CommandHandler> handlers){
List<Completer> completers = new LinkedList<Completer>();
List<Completer> completers = new LinkedList<>();

for (CommandHandler handler : handlers) {
String[] commandNames = handler.getNames();
if (commandNames != null) {
for (String commandName : commandNames) {
List<Completer> compl = new LinkedList<Completer>();
compl.add(new StringsCompleter(BeeLine.COMMAND_PREFIX + commandName));
List<Completer> compl = new LinkedList<>();
compl.add(new MatchingStringsCompleter(BeeLine.COMMAND_PREFIX + commandName));
compl.addAll(Arrays.asList(handler.getParameterCompleters()));
compl.add(new NullCompleter()); // last param no complete
completers.add(new AggregateCompleter(compl.toArray(new Completer[compl.size()])));
completers.add(new AggregateCompleter(compl.toArray(new Completer[0])));
}
}
}

return completers;
}
}
}
Loading