diff --git LICENSE LICENSE index 2885945..546eca2 100644 --- LICENSE +++ LICENSE @@ -170,7 +170,7 @@ License. However, in accepting such obligations, You may act only on Your own behalf and on Your sole responsibility, not on behalf of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability + defend, and hold each Contributor harmless for any liability contains incurred by, or claims asserted against, such Contributor by reason of your accepting any such warranty or additional liability. @@ -480,4 +480,39 @@ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. - + + +For the org.apache.hive.beeline.ClassNameCompleter class: + +Copyright (c) 2002-2006, Marc Prud'hommeaux +All rights reserved. + +Redistribution and use in source and binary forms, with or +without modification, are permitted provided that the following +conditions are met: + +Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + +Redistributions in binary form must reproduce the above copyright +notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with +the distribution. + +Neither the name of JLine nor the names of its contributors +may be used to endorse or promote products derived from this +software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, +BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY +AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO +EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, +OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED +AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT +LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING +IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED +OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git beeline/src/java/org/apache/hive/beeline/AbstractCommandHandler.java beeline/src/java/org/apache/hive/beeline/AbstractCommandHandler.java index a9479d5..625879b 100644 --- beeline/src/java/org/apache/hive/beeline/AbstractCommandHandler.java +++ beeline/src/java/org/apache/hive/beeline/AbstractCommandHandler.java @@ -26,8 +26,8 @@ import java.util.LinkedList; import java.util.List; -import jline.Completor; -import jline.NullCompletor; +import jline.console.completer.Completer; +import jline.console.completer.NullCompleter; /** * An abstract implementation of CommandHandler. @@ -38,22 +38,22 @@ private final String name; private final String[] names; private final String helpText; - private Completor[] parameterCompletors = new Completor[0]; + private Completer[] parameterCompleters = new Completer[0]; protected transient Throwable lastException; public AbstractCommandHandler(BeeLine beeLine, String[] names, String helpText, - Completor[] completors) { + Completer[] completors) { this.beeLine = beeLine; name = names[0]; this.names = names; this.helpText = helpText; if (completors == null || completors.length == 0) { - parameterCompletors = new Completor[] { new NullCompletor() }; + parameterCompleters = new Completer[] { new NullCompleter() }; } else { - List c = new LinkedList(Arrays.asList(completors)); - c.add(new NullCompletor()); - parameterCompletors = c.toArray(new Completor[0]); + List c = new LinkedList(Arrays.asList(completors)); + c.add(new NullCompleter()); + parameterCompleters = c.toArray(new Completer[0]); } } @@ -94,13 +94,13 @@ public String matches(String line) { return null; } - public void setParameterCompletors(Completor[] parameterCompletors) { - this.parameterCompletors = parameterCompletors; + public void setParameterCompleters(Completer[] parameterCompleters) { + this.parameterCompleters = parameterCompleters; } @Override - public Completor[] getParameterCompletors() { - return parameterCompletors; + public Completer[] getParameterCompleters() { + return parameterCompleters; } @Override diff --git beeline/src/java/org/apache/hive/beeline/BeeLine.java beeline/src/java/org/apache/hive/beeline/BeeLine.java index 8539a41..630ead4 100644 --- beeline/src/java/org/apache/hive/beeline/BeeLine.java +++ beeline/src/java/org/apache/hive/beeline/BeeLine.java @@ -72,13 +72,14 @@ import java.util.zip.ZipEntry; import java.util.zip.ZipFile; -import jline.ClassNameCompletor; -import jline.Completor; -import jline.ConsoleReader; -import jline.FileNameCompletor; -import jline.History; -import jline.SimpleCompletor; - +import jline.console.completer.Completer; +import jline.console.completer.StringsCompleter; +import jline.console.completer.FileNameCompleter; +import jline.console.ConsoleReader; +import jline.console.history.History; +import jline.console.history.FileHistory; + +import jline.internal.Log; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.OptionBuilder; @@ -111,7 +112,7 @@ private boolean exit = false; private final DatabaseConnections connections = new DatabaseConnections(); public static final String COMMAND_PREFIX = "!"; - private final Completor beeLineCommandCompletor; + private final Completer beeLineCommandCompleter; private Collection drivers = null; private final BeeLineOpts opts = new BeeLineOpts(this, System.getProperties()); private String lastProgress = null; @@ -159,19 +160,19 @@ new ReflectiveCommandHandler(this, new String[] {"quit", "done", "exit"}, null), new ReflectiveCommandHandler(this, new String[] {"connect", "open"}, - new Completor[] {new SimpleCompletor(getConnectionURLExamples())}), + new Completer[] {new StringsCompleter(getConnectionURLExamples())}), new ReflectiveCommandHandler(this, new String[] {"describe"}, - new Completor[] {new TableNameCompletor(this)}), + new Completer[] {new TableNameCompletor(this)}), new ReflectiveCommandHandler(this, new String[] {"indexes"}, - new Completor[] {new TableNameCompletor(this)}), + new Completer[] {new TableNameCompletor(this)}), new ReflectiveCommandHandler(this, new String[] {"primarykeys"}, - new Completor[] {new TableNameCompletor(this)}), + new Completer[] {new TableNameCompletor(this)}), new ReflectiveCommandHandler(this, new String[] {"exportedkeys"}, - new Completor[] {new TableNameCompletor(this)}), + new Completer[] {new TableNameCompletor(this)}), new ReflectiveCommandHandler(this, new String[] {"manual"}, null), new ReflectiveCommandHandler(this, new String[] {"importedkeys"}, - new Completor[] {new TableNameCompletor(this)}), + new Completer[] {new TableNameCompletor(this)}), new ReflectiveCommandHandler(this, new String[] {"procedures"}, null), new ReflectiveCommandHandler(this, new String[] {"tables"}, @@ -179,16 +180,16 @@ new ReflectiveCommandHandler(this, new String[] {"typeinfo"}, null), new ReflectiveCommandHandler(this, new String[] {"columns"}, - new Completor[] {new TableNameCompletor(this)}), + new Completer[] {new TableNameCompletor(this)}), new ReflectiveCommandHandler(this, new String[] {"reconnect"}, null), new ReflectiveCommandHandler(this, new String[] {"dropall"}, - new Completor[] {new TableNameCompletor(this)}), + new Completer[] {new TableNameCompletor(this)}), new ReflectiveCommandHandler(this, new String[] {"history"}, null), new ReflectiveCommandHandler(this, new String[] {"metadata"}, - new Completor[] { - new SimpleCompletor(getMetadataMethodNames())}), + new Completer[] { + new StringsCompleter(getMetadataMethodNames())}), new ReflectiveCommandHandler(this, new String[] {"nativesql"}, null), new ReflectiveCommandHandler(this, new String[] {"dbinfo"}, @@ -198,7 +199,7 @@ new ReflectiveCommandHandler(this, new String[] {"verbose"}, null), new ReflectiveCommandHandler(this, new String[] {"run"}, - new Completor[] {new FileNameCompletor()}), + new Completer[] {new FileNameCompleter()}), new ReflectiveCommandHandler(this, new String[] {"batch"}, null), new ReflectiveCommandHandler(this, new String[] {"list"}, @@ -208,9 +209,9 @@ new ReflectiveCommandHandler(this, new String[] {"go", "#"}, null), new ReflectiveCommandHandler(this, new String[] {"script"}, - new Completor[] {new FileNameCompletor()}), + new Completer[] {new FileNameCompleter()}), new ReflectiveCommandHandler(this, new String[] {"record"}, - new Completor[] {new FileNameCompletor()}), + new Completer[] {new FileNameCompleter()}), new ReflectiveCommandHandler(this, new String[] {"brief"}, null), new ReflectiveCommandHandler(this, new String[] {"close"}, @@ -218,22 +219,22 @@ new ReflectiveCommandHandler(this, new String[] {"closeall"}, null), new ReflectiveCommandHandler(this, new String[] {"isolation"}, - new Completor[] {new SimpleCompletor(getIsolationLevels())}), + new Completer[] {new StringsCompleter(getIsolationLevels())}), new ReflectiveCommandHandler(this, new String[] {"outputformat"}, - new Completor[] {new SimpleCompletor( + new Completer[] {new StringsCompleter( formats.keySet().toArray(new String[0]))}), new ReflectiveCommandHandler(this, new String[] {"autocommit"}, null), new ReflectiveCommandHandler(this, new String[] {"commit"}, null), new ReflectiveCommandHandler(this, new String[] {"properties"}, - new Completor[] {new FileNameCompletor()}), + new Completer[] {new FileNameCompleter()}), new ReflectiveCommandHandler(this, new String[] {"rollback"}, null), new ReflectiveCommandHandler(this, new String[] {"help", "?"}, null), new ReflectiveCommandHandler(this, new String[] {"set"}, - getOpts().optionCompletors()), + getOpts().optionCompleters()), new ReflectiveCommandHandler(this, new String[] {"save"}, null), new ReflectiveCommandHandler(this, new String[] {"scan"}, @@ -245,7 +246,7 @@ new ReflectiveCommandHandler(this, new String[] {"call"}, null), new ReflectiveCommandHandler(this, new String[] {"nullemptystring"}, - new Completor[] {new BooleanCompletor()}), + new Completer[] {new BooleanCompleter()}), }; @@ -258,7 +259,7 @@ static { try { - Class.forName("jline.ConsoleReader"); + Class.forName("jline.console.ConsoleReader"); } catch (Throwable t) { throw new ExceptionInInitializerError("jline-missing"); } @@ -469,13 +470,14 @@ public static void mainWithInputRedirection(String[] args, InputStream inputStre int status = beeLine.begin(args, inputStream); if (!Boolean.getBoolean(BeeLineOpts.PROPERTY_NAME_EXIT)) { - System.exit(status); + System.exit(status); } } public BeeLine() { - beeLineCommandCompletor = new BeeLineCommandCompletor(this); + beeLineCommandCompleter = new BeeLineCommandCompleter(BeeLineCommandCompleter.getCompleters + (this)); reflector = new Reflector(this); // attempt to dynamically load signal handler @@ -737,7 +739,7 @@ public int begin(String[] args, InputStream inputStream) throws IOException { ConsoleReader reader = getConsoleReader(inputStream); return execute(reader, false); } finally { - close(); + close(); } } @@ -792,14 +794,16 @@ public void close() { public ConsoleReader getConsoleReader(InputStream inputStream) throws IOException { if (inputStream != null) { // ### NOTE: fix for sf.net bug 879425. - consoleReader = new ConsoleReader(inputStream, new PrintWriter(getOutputStream(), true)); + consoleReader = new ConsoleReader(inputStream, getOutputStream()); } else { consoleReader = new ConsoleReader(); } - // setup history - ByteArrayInputStream historyBuffer = null; + //disable the expandEvents for the purpose of backward compatibility + consoleReader.setExpandEvents(false); + // setup history + ByteArrayOutputStream hist = null; if (new File(getOpts().getHistoryFile()).isFile()) { try { // save the current contents of the history buffer. This gets @@ -808,13 +812,12 @@ public ConsoleReader getConsoleReader(InputStream inputStream) throws IOExceptio // input before the output will cause the previous commands // to not be saved to the buffer. FileInputStream historyIn = new FileInputStream(getOpts().getHistoryFile()); - ByteArrayOutputStream hist = new ByteArrayOutputStream(); + hist = new ByteArrayOutputStream(); int n; while ((n = historyIn.read()) != -1) { hist.write(n); } historyIn.close(); - historyBuffer = new ByteArrayInputStream(hist.toByteArray()); } catch (Exception e) { handleException(e); } @@ -822,25 +825,46 @@ public ConsoleReader getConsoleReader(InputStream inputStream) throws IOExceptio try { // now set the output for the history - PrintWriter historyOut = new PrintWriter(new FileWriter(getOpts().getHistoryFile()), true); - consoleReader.getHistory().setOutput(historyOut); + consoleReader.setHistory(new FileHistory(new File(getOpts().getHistoryFile()))); } catch (Exception e) { handleException(e); } if (inputStream instanceof FileInputStream) { - // from script.. no need to load history and no need of completor, either + // from script.. no need to load history and no need of completer, either return consoleReader; } try { // now load in the previous history - if (historyBuffer != null) { - consoleReader.getHistory().load(historyBuffer); + if (hist != null) { + History h = consoleReader.getHistory(); + if (h instanceof FileHistory) { + ((FileHistory) consoleReader.getHistory()).load(new ByteArrayInputStream(hist + .toByteArray())); + } else { + consoleReader.getHistory().add(hist.toString()); + } } } catch (Exception e) { - handleException(e); + handleException(e); } - consoleReader.addCompletor(new BeeLineCompletor(this)); + + // add shutdown hook to flush the history to history file + Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { + @Override + public void run() { + History h = consoleReader.getHistory(); + if (h instanceof FileHistory) { + try { + ((FileHistory) h).flush(); + } catch (IOException e) { + error(e); + } + } + } + })); + + consoleReader.addCompleter(new BeeLineCompleter(this)); return consoleReader; } @@ -1563,7 +1587,7 @@ private Driver findRegisteredDriver(String url) { if (!knownOnly) { classNames.addAll(Arrays.asList( - ClassNameCompletor.getClassNames())); + ClassNameCompleter.getClassNames())); } classNames.addAll(KNOWN_DRIVERS); @@ -1782,8 +1806,8 @@ DatabaseConnections getDatabaseConnections() { return connections; } - Completor getCommandCompletor() { - return beeLineCommandCompletor; + Completer getCommandCompletor() { + return beeLineCommandCompleter; } public boolean isExit() { diff --git beeline/src/java/org/apache/hive/beeline/BeeLineCommandCompleter.java beeline/src/java/org/apache/hive/beeline/BeeLineCommandCompleter.java new file mode 100644 index 0000000..6a872bc --- /dev/null +++ beeline/src/java/org/apache/hive/beeline/BeeLineCommandCompleter.java @@ -0,0 +1,50 @@ +/** + * 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; + +import java.util.Arrays; +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; + +class BeeLineCommandCompleter extends AggregateCompleter { + + public BeeLineCommandCompleter(List completers) { + super(completers); + } + + public static List getCompleters(BeeLine beeLine){ + List completers = new LinkedList(); + + for (int i = 0; i < beeLine.commandHandlers.length; i++) { + String[] cmds = beeLine.commandHandlers[i].getNames(); + for (int j = 0; cmds != null && j < cmds.length; j++) { + List compl = new LinkedList(); + compl.add(new StringsCompleter(BeeLine.COMMAND_PREFIX + cmds[j])); + compl.addAll(Arrays.asList(beeLine.commandHandlers[i].getParameterCompleters())); + compl.add(new NullCompleter()); // last param no complete + completers.add(new AggregateCompleter(compl.toArray(new Completer[0]))); + } + } + return completers; + } +} \ No newline at end of file diff --git beeline/src/java/org/apache/hive/beeline/BeeLineCommandCompletor.java beeline/src/java/org/apache/hive/beeline/BeeLineCommandCompletor.java deleted file mode 100644 index 52313e6..0000000 --- beeline/src/java/org/apache/hive/beeline/BeeLineCommandCompletor.java +++ /dev/null @@ -1,51 +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; - -import java.util.Arrays; -import java.util.LinkedList; -import java.util.List; - -import jline.ArgumentCompletor; -import jline.Completor; -import jline.MultiCompletor; -import jline.NullCompletor; -import jline.SimpleCompletor; - -class BeeLineCommandCompletor extends MultiCompletor { - private final BeeLine beeLine; - - public BeeLineCommandCompletor(BeeLine beeLine) { - this.beeLine = beeLine; - List completors = new LinkedList(); - - for (int i = 0; i < beeLine.commandHandlers.length; i++) { - String[] cmds = beeLine.commandHandlers[i].getNames(); - for (int j = 0; cmds != null && j < cmds.length; j++) { - Completor[] comps = beeLine.commandHandlers[i].getParameterCompletors(); - List compl = new LinkedList(); - compl.add(new SimpleCompletor(BeeLine.COMMAND_PREFIX + cmds[j])); - compl.addAll(Arrays.asList(comps)); - compl.add(new NullCompletor()); // last param no complete - completors.add(new ArgumentCompletor( - compl.toArray(new Completor[0]))); - } - } - setCompletors(completors.toArray(new Completor[0])); - } -} \ No newline at end of file diff --git beeline/src/java/org/apache/hive/beeline/BeeLineCompleter.java beeline/src/java/org/apache/hive/beeline/BeeLineCompleter.java new file mode 100644 index 0000000..87f0150 --- /dev/null +++ beeline/src/java/org/apache/hive/beeline/BeeLineCompleter.java @@ -0,0 +1,57 @@ +/** + * 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. + */ + +/* + * This source file is based on code taken from SQLLine 1.0.2 + * See SQLLine notice in LICENSE + */ +package org.apache.hive.beeline; + +import java.util.List; + +import jline.console.completer.Completer; + +/** + * Completor for BeeLine. It dispatches to sub-completors based on the + * current arguments. + * + */ +class BeeLineCompleter implements Completer { + private final BeeLine beeLine; + + /** + * @param beeLine + */ + BeeLineCompleter(BeeLine beeLine) { + this.beeLine = beeLine; + } + + public int complete(String buf, int pos, List cand) { + if (buf != null && buf.startsWith(BeeLine.COMMAND_PREFIX) + && !buf.startsWith(BeeLine.COMMAND_PREFIX + "all") + && !buf.startsWith(BeeLine.COMMAND_PREFIX + "sql")) { + return beeLine.getCommandCompletor().complete(buf, pos, cand); + } else { + if (beeLine.getDatabaseConnection() != null && beeLine.getDatabaseConnection().getSQLCompleter() != null) { + return beeLine.getDatabaseConnection().getSQLCompleter().complete(buf, pos, cand); + } else { + return -1; + } + } + } +} \ No newline at end of file diff --git beeline/src/java/org/apache/hive/beeline/BeeLineCompletor.java beeline/src/java/org/apache/hive/beeline/BeeLineCompletor.java deleted file mode 100644 index c6bb4fe..0000000 --- beeline/src/java/org/apache/hive/beeline/BeeLineCompletor.java +++ /dev/null @@ -1,57 +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. - */ - -/* - * This source file is based on code taken from SQLLine 1.0.2 - * See SQLLine notice in LICENSE - */ -package org.apache.hive.beeline; - -import java.util.List; - -import jline.Completor; - -/** - * Completor for BeeLine. It dispatches to sub-completors based on the - * current arguments. - * - */ -class BeeLineCompletor implements Completor { - private final BeeLine beeLine; - - /** - * @param beeLine - */ - BeeLineCompletor(BeeLine beeLine) { - this.beeLine = beeLine; - } - - public int complete(String buf, int pos, List cand) { - if (buf != null && buf.startsWith(BeeLine.COMMAND_PREFIX) - && !buf.startsWith(BeeLine.COMMAND_PREFIX + "all") - && !buf.startsWith(BeeLine.COMMAND_PREFIX + "sql")) { - return beeLine.getCommandCompletor().complete(buf, pos, cand); - } else { - if (beeLine.getDatabaseConnection() != null && beeLine.getDatabaseConnection().getSQLCompletor() != null) { - return beeLine.getDatabaseConnection().getSQLCompletor().complete(buf, pos, cand); - } else { - return -1; - } - } - } -} \ No newline at end of file diff --git beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java index f73fb44..649bb63 100644 --- beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java +++ beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java @@ -38,12 +38,12 @@ import java.util.Properties; import java.util.TreeSet; -import jline.Completor; -import jline.SimpleCompletor; import jline.Terminal; +import jline.TerminalFactory; +import jline.console.completer.Completer; +import jline.console.completer.StringsCompleter; - -class BeeLineOpts implements Completor { +class BeeLineOpts implements Completer { public static final int DEFAULT_MAX_WIDTH = 80; public static final int DEFAULT_MAX_HEIGHT = 80; public static final int DEFAULT_HEADER_INTERVAL = 100; @@ -70,7 +70,7 @@ private boolean showNestedErrs = false; private boolean showElapsedTime = true; private String numberFormat = "default"; - private final Terminal terminal = Terminal.setupTerminal(); + private final Terminal terminal = TerminalFactory.get(); private int maxWidth = DEFAULT_MAX_WIDTH; private int maxHeight = DEFAULT_MAX_HEIGHT; private int maxColumnWidth = 15; @@ -98,18 +98,17 @@ public BeeLineOpts(BeeLine beeLine, Properties props) { this.beeLine = beeLine; - if (terminal.getTerminalWidth() > 0) { - maxWidth = terminal.getTerminalWidth(); + if (terminal.getWidth() > 0) { + maxWidth = terminal.getWidth(); } - if (terminal.getTerminalHeight() > 0) { - maxHeight = terminal.getTerminalHeight(); + if (terminal.getHeight() > 0) { + maxHeight = terminal.getHeight(); } loadProperties(props); } - - public Completor[] optionCompletors() { - return new Completor[] {this}; + public Completer[] optionCompleters() { + return new Completer[] {this}; } public String[] possibleSettingValues() { @@ -144,7 +143,7 @@ public File saveDir() { @Override public int complete(String buf, int pos, List cand) { try { - return new SimpleCompletor(propertyNames()).complete(buf, pos, cand); + return new StringsCompleter(propertyNames()).complete(buf, pos, cand); } catch (Exception e) { beeLine.handleException(e); return -1; diff --git beeline/src/java/org/apache/hive/beeline/BooleanCompleter.java beeline/src/java/org/apache/hive/beeline/BooleanCompleter.java new file mode 100644 index 0000000..47fe70e --- /dev/null +++ beeline/src/java/org/apache/hive/beeline/BooleanCompleter.java @@ -0,0 +1,31 @@ +/** + * 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; + +import jline.console.completer.StringsCompleter; + +/** + * JLine completor boolean value (true/false) + */ +class BooleanCompleter extends StringsCompleter { + + public BooleanCompleter(){ + super(new String[] {"true", "false"}); + } + +} \ No newline at end of file diff --git beeline/src/java/org/apache/hive/beeline/BooleanCompletor.java beeline/src/java/org/apache/hive/beeline/BooleanCompletor.java deleted file mode 100644 index 3e88c53..0000000 --- beeline/src/java/org/apache/hive/beeline/BooleanCompletor.java +++ /dev/null @@ -1,36 +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; - -import java.util.List; - -import jline.Completor; -import jline.SimpleCompletor; - -/** - * JLine completor boolean value (true/false) - */ -class BooleanCompletor extends SimpleCompletor { - - public BooleanCompletor(){ - super(new String[] {"true", "false"}); - } - -} \ No newline at end of file diff --git beeline/src/java/org/apache/hive/beeline/ClassNameCompleter.java beeline/src/java/org/apache/hive/beeline/ClassNameCompleter.java new file mode 100644 index 0000000..065eab4 --- /dev/null +++ beeline/src/java/org/apache/hive/beeline/ClassNameCompleter.java @@ -0,0 +1,184 @@ +/** + * Copyright (c) 2002-2006, Marc Prud'hommeaux + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or + * without modification, are permitted provided that the following + * conditions are met: + * + * Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * + * Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer + * in the documentation and/or other materials provided with + * the distribution. + * + * Neither the name of JLine nor the names of its contributors + * may be used to endorse or promote products derived from this + * software without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, + * BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY + * AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO + * EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, + * OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, + * PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED + * AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT + * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING + * IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED + * OF THE POSSIBILITY OF SUCH DAMAGE. + */ +/** + * 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; + +import jline.console.completer.StringsCompleter; + +import java.io.File; +import java.io.IOException; +import java.net.JarURLConnection; +import java.net.URL; +import java.net.URLClassLoader; +import java.net.URLConnection; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; +import java.util.jar.JarEntry; +import java.util.jar.JarFile; +import java.util.Enumeration; +import java.util.TreeSet; + +/** + * the completer is original provided in JLine 0.9.94 and is being removed in 2.12. Add the + * previous implement for usage of the beeline. + */ +public class ClassNameCompleter extends StringsCompleter { + + public ClassNameCompleter(String... candidates) { + super(candidates); + } + + public static String[] getClassNames() throws IOException { + Set urls = new HashSet(); + + for (ClassLoader loader = ClassNameCompleter.class.getClassLoader(); loader != null; + loader = loader.getParent()) { + if (!(loader instanceof URLClassLoader)) { + continue; + } + + urls.addAll(Arrays.asList(((URLClassLoader) loader).getURLs())); + } + + // Now add the URL that holds java.lang.String. This is because + // some JVMs do not report the core classes jar in the list of + // class loaders. + Class[] systemClasses = new Class[]{String.class, javax.swing.JFrame.class}; + + for (int i = 0; i < systemClasses.length; i++) { + URL classURL = systemClasses[i] + .getResource("/" + systemClasses[i].getName().replace('.', '/') + ".class"); + + if (classURL != null) { + URLConnection uc = classURL.openConnection(); + + if (uc instanceof JarURLConnection) { + urls.add(((JarURLConnection) uc).getJarFileURL()); + } + } + } + + Set classes = new HashSet(); + + for (Iterator i = urls.iterator(); i.hasNext(); ) { + URL url = (URL) i.next(); + File file = new File(url.getFile()); + + if (file.isDirectory()) { + Set files = getClassFiles(file.getAbsolutePath(), new HashSet(), file, new int[]{200}); + classes.addAll(files); + + continue; + } + + if ((file == null) || !file.isFile()) { + continue; + } + + JarFile jf = new JarFile(file); + + for (Enumeration e = jf.entries(); e.hasMoreElements(); ) { + JarEntry entry = (JarEntry) e.nextElement(); + + if (entry == null) { + continue; + } + + String name = entry.getName(); + + if (!name.endsWith(".class")) { + /* only use class file*/ + continue; + } + + classes.add(name); + } + } + + // now filter classes by changing "/" to "." and trimming the + // trailing ".class" + Set classNames = new TreeSet(); + + for (Iterator i = classes.iterator(); i.hasNext(); ) { + String name = (String) i.next(); + classNames.add(name.replace('/', '.'). + substring(0, name.length() - 6)); + } + + return (String[]) classNames.toArray(new String[classNames.size()]); + } + + private static Set getClassFiles(String root, Set holder, File directory, int[] maxDirectories) { + // we have passed the maximum number of directories to scan + if (maxDirectories[0]-- < 0) { + return holder; + } + + File[] files = directory.listFiles(); + + for (int i = 0; (files != null) && (i < files.length); i++) { + String name = files[i].getAbsolutePath(); + + if (!(name.startsWith(root))) { + continue; + } else if (files[i].isDirectory()) { + getClassFiles(root, holder, files[i], maxDirectories); + } else if (files[i].getName().endsWith(".class")) { + holder.add(files[i].getAbsolutePath(). + substring(root.length() + 1)); + } + } + + return holder; + } +} diff --git beeline/src/java/org/apache/hive/beeline/CommandHandler.java beeline/src/java/org/apache/hive/beeline/CommandHandler.java index bab1778..e0080df 100644 --- beeline/src/java/org/apache/hive/beeline/CommandHandler.java +++ beeline/src/java/org/apache/hive/beeline/CommandHandler.java @@ -22,7 +22,7 @@ */ package org.apache.hive.beeline; -import jline.Completor; +import jline.console.completer.Completer; /** * A generic command to be executed. Execution of the command @@ -73,7 +73,7 @@ /** * Returns the completors that can handle parameters. */ - public Completor[] getParameterCompletors(); + public Completer[] getParameterCompleters(); /** * Returns exception thrown for last command diff --git beeline/src/java/org/apache/hive/beeline/Commands.java beeline/src/java/org/apache/hive/beeline/Commands.java index 7e366dc..0fa2821 100644 --- beeline/src/java/org/apache/hive/beeline/Commands.java +++ beeline/src/java/org/apache/hive/beeline/Commands.java @@ -128,11 +128,11 @@ public boolean metadata(String cmd, String[] args) { public boolean history(String line) { - List hist = beeLine.getConsoleReader().getHistory().getHistoryList(); + Iterator hist = beeLine.getConsoleReader().getHistory().entries(); int index = 1; - for (Iterator i = hist.iterator(); i.hasNext(); index++) { + while(hist.hasNext()){ beeLine.output(beeLine.getColorBuffer().pad(index + ".", 6) - .append(i.next().toString())); + .append(hist.next().toString())); } return true; } diff --git beeline/src/java/org/apache/hive/beeline/DatabaseConnection.java beeline/src/java/org/apache/hive/beeline/DatabaseConnection.java index ab67700..8ba0232 100644 --- beeline/src/java/org/apache/hive/beeline/DatabaseConnection.java +++ beeline/src/java/org/apache/hive/beeline/DatabaseConnection.java @@ -35,8 +35,8 @@ import java.util.Set; import java.util.TreeSet; -import jline.ArgumentCompletor; -import jline.Completor; +import jline.console.completer.ArgumentCompleter; +import jline.console.completer.Completer; class DatabaseConnection { private static final String HIVE_AUTH_USER = "user"; @@ -51,7 +51,7 @@ private final String url; private final Properties info; private Schema schema = null; - private Completor sqlCompletor = null; + private Completer sqlCompleter = null; public boolean isClosed() { return (null == connection); @@ -76,17 +76,16 @@ void setCompletions(boolean skipmeta) throws SQLException, IOException { getDatabaseMetaData() == null || getDatabaseMetaData().getExtraNameCharacters() == null ? "" : getDatabaseMetaData().getExtraNameCharacters(); - // setup the completor for the database - sqlCompletor = new ArgumentCompletor( - new SQLCompletor(beeLine, skipmeta), - new ArgumentCompletor.AbstractArgumentDelimiter() { + // setup the completer for the database + sqlCompleter = new ArgumentCompleter( + new ArgumentCompleter.AbstractArgumentDelimiter() { // delimiters for SQL statements are any // non-letter-or-number characters, except // underscore and characters that are specified // by the database to be valid name identifiers. @Override - public boolean isDelimiterChar(String buf, int pos) { - char c = buf.charAt(pos); + public boolean isDelimiterChar(CharSequence buffer, int pos) { + char c = buffer.charAt(pos); if (Character.isWhitespace(c)) { return true; } @@ -94,10 +93,10 @@ public boolean isDelimiterChar(String buf, int pos) { && c != '_' && extraNameCharacters.indexOf(c) == -1; } - }); - + }, + new SQLCompleter(SQLCompleter.getSQLCompleters(beeLine, skipmeta))); // not all argument elements need to hold true - ((ArgumentCompletor) sqlCompletor).setStrict(false); + ((ArgumentCompleter) sqlCompleter).setStrict(false); } @@ -236,8 +235,8 @@ String getUrl() { return url; } - Completor getSQLCompletor() { - return sqlCompletor; + Completer getSQLCompleter() { + return sqlCompleter; } class Schema { diff --git beeline/src/java/org/apache/hive/beeline/ReflectiveCommandHandler.java beeline/src/java/org/apache/hive/beeline/ReflectiveCommandHandler.java index 2b957f2..3b863ae 100644 --- beeline/src/java/org/apache/hive/beeline/ReflectiveCommandHandler.java +++ beeline/src/java/org/apache/hive/beeline/ReflectiveCommandHandler.java @@ -22,7 +22,7 @@ */ package org.apache.hive.beeline; -import jline.Completor; +import jline.console.completer.Completer; import org.apache.hadoop.fs.shell.Command; @@ -34,8 +34,15 @@ public class ReflectiveCommandHandler extends AbstractCommandHandler { private final BeeLine beeLine; - public ReflectiveCommandHandler(BeeLine beeLine, String[] cmds, Completor[] completor) { - super(beeLine, cmds, beeLine.loc("help-" + cmds[0]), completor); + /** + * @param beeLine + * @param cmds 'cmds' is an array of alternative names for the same command. And that the + * first one is always chosen for display purposes and to lookup help + * documentation from BeeLine.properties file. + * @param completer + */ + public ReflectiveCommandHandler(BeeLine beeLine, String[] cmds, Completer[] completer) { + super(beeLine, cmds, beeLine.loc("help-" + cmds[0]), completer); this.beeLine = beeLine; } diff --git beeline/src/java/org/apache/hive/beeline/SQLCompleter.java beeline/src/java/org/apache/hive/beeline/SQLCompleter.java new file mode 100644 index 0000000..3f2f7b0 --- /dev/null +++ beeline/src/java/org/apache/hive/beeline/SQLCompleter.java @@ -0,0 +1,90 @@ +/** + * 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. + */ + +/* + * This source file is based on code taken from SQLLine 1.0.2 + * See SQLLine notice in LICENSE + */ +package org.apache.hive.beeline; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.sql.SQLException; +import java.util.Set; +import java.util.StringTokenizer; +import java.util.TreeSet; + +import jline.console.completer.StringsCompleter; + +class SQLCompleter extends StringsCompleter { + + public SQLCompleter(Set completions){ + super(completions); + } + + public static Set getSQLCompleters(BeeLine beeLine, boolean skipmeta) + throws IOException, SQLException { + Set completions = new TreeSet(); + + // add the default SQL completions + String keywords = new BufferedReader(new InputStreamReader( + SQLCompleter.class.getResourceAsStream( + "/sql-keywords.properties"))).readLine(); + + // now add the keywords from the current connection + try { + keywords += "," + beeLine.getDatabaseConnection().getDatabaseMetaData().getSQLKeywords(); + } catch (Throwable t) { + } + try { + keywords += "," + beeLine.getDatabaseConnection().getDatabaseMetaData().getStringFunctions(); + } catch (Throwable t) { + } + try { + keywords += "," + beeLine.getDatabaseConnection().getDatabaseMetaData().getNumericFunctions(); + } catch (Throwable t) { + } + try { + keywords += "," + beeLine.getDatabaseConnection().getDatabaseMetaData().getSystemFunctions(); + } catch (Throwable t) { + } + try { + keywords += "," + beeLine.getDatabaseConnection().getDatabaseMetaData().getTimeDateFunctions(); + } catch (Throwable t) { + } + + // also allow lower-case versions of all the keywords + keywords += "," + keywords.toLowerCase(); + + for (StringTokenizer tok = new StringTokenizer(keywords, ", "); tok.hasMoreTokens(); completions + .add(tok.nextToken())) { + ; + } + + // now add the tables and columns from the current connection + if (!(skipmeta)) { + String[] columns = beeLine.getColumnNames(beeLine.getDatabaseConnection().getDatabaseMetaData()); + for (int i = 0; columns != null && i < columns.length; i++) { + completions.add(columns[i++]); + } + } + + return completions; + } +} diff --git beeline/src/java/org/apache/hive/beeline/SQLCompletor.java beeline/src/java/org/apache/hive/beeline/SQLCompletor.java deleted file mode 100644 index 844b9ae..0000000 --- beeline/src/java/org/apache/hive/beeline/SQLCompletor.java +++ /dev/null @@ -1,91 +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. - */ - -/* - * This source file is based on code taken from SQLLine 1.0.2 - * See SQLLine notice in LICENSE - */ -package org.apache.hive.beeline; - -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStreamReader; -import java.sql.SQLException; -import java.util.Set; -import java.util.StringTokenizer; -import java.util.TreeSet; - -import jline.SimpleCompletor; - -class SQLCompletor extends SimpleCompletor { - private final BeeLine beeLine; - - public SQLCompletor(BeeLine beeLine, boolean skipmeta) - throws IOException, SQLException { - super(new String[0]); - this.beeLine = beeLine; - - Set completions = new TreeSet(); - - // add the default SQL completions - String keywords = new BufferedReader(new InputStreamReader( - SQLCompletor.class.getResourceAsStream( - "/sql-keywords.properties"))).readLine(); - - // now add the keywords from the current connection - try { - keywords += "," + beeLine.getDatabaseConnection().getDatabaseMetaData().getSQLKeywords(); - } catch (Throwable t) { - } - try { - keywords += "," + beeLine.getDatabaseConnection().getDatabaseMetaData().getStringFunctions(); - } catch (Throwable t) { - } - try { - keywords += "," + beeLine.getDatabaseConnection().getDatabaseMetaData().getNumericFunctions(); - } catch (Throwable t) { - } - try { - keywords += "," + beeLine.getDatabaseConnection().getDatabaseMetaData().getSystemFunctions(); - } catch (Throwable t) { - } - try { - keywords += "," + beeLine.getDatabaseConnection().getDatabaseMetaData().getTimeDateFunctions(); - } catch (Throwable t) { - } - - // also allow lower-case versions of all the keywords - keywords += "," + keywords.toLowerCase(); - - for (StringTokenizer tok = new StringTokenizer(keywords, ", "); tok.hasMoreTokens(); completions - .add(tok.nextToken())) { - ; - } - - // now add the tables and columns from the current connection - if (!(skipmeta)) { - String[] columns = beeLine.getColumnNames(beeLine.getDatabaseConnection().getDatabaseMetaData()); - for (int i = 0; columns != null && i < columns.length; i++) { - completions.add(columns[i++]); - } - } - - // set the Strings that will be completed - setCandidateStrings(completions.toArray(new String[0])); - } -} diff --git beeline/src/java/org/apache/hive/beeline/TableNameCompletor.java beeline/src/java/org/apache/hive/beeline/TableNameCompletor.java index bc0d9be..663db28 100644 --- beeline/src/java/org/apache/hive/beeline/TableNameCompletor.java +++ beeline/src/java/org/apache/hive/beeline/TableNameCompletor.java @@ -24,10 +24,10 @@ import java.util.List; -import jline.Completor; -import jline.SimpleCompletor; +import jline.console.completer.Completer; +import jline.console.completer.StringsCompleter; -class TableNameCompletor implements Completor { +class TableNameCompletor implements Completer { private final BeeLine beeLine; /** @@ -41,7 +41,7 @@ public int complete(String buf, int pos, List cand) { if (beeLine.getDatabaseConnection() == null) { return -1; } - return new SimpleCompletor(beeLine.getDatabaseConnection().getTableNames(true)) + return new StringsCompleter(beeLine.getDatabaseConnection().getTableNames(true)) .complete(buf, pos, cand); } } \ No newline at end of file diff --git cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java index 0ccaacb..ebe7ae9 100644 --- cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java +++ cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java @@ -29,17 +29,16 @@ import java.io.UnsupportedEncodingException; import java.sql.SQLException; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; -import jline.ArgumentCompletor; -import jline.ArgumentCompletor.AbstractArgumentDelimiter; -import jline.ArgumentCompletor.ArgumentDelimiter; -import jline.Completor; -import jline.ConsoleReader; -import jline.History; -import jline.SimpleCompletor; +import jline.console.ConsoleReader; +import jline.console.completer.Completer; +import jline.console.history.FileHistory; +import jline.console.completer.StringsCompleter; +import jline.console.completer.ArgumentCompleter; +import jline.console.completer.ArgumentCompleter.ArgumentDelimiter; +import jline.console.completer.ArgumentCompleter.AbstractArgumentDelimiter; import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; @@ -58,7 +57,6 @@ import org.apache.hadoop.hive.ql.CommandNeedRetryException; import org.apache.hadoop.hive.ql.Driver; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; -import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.mr.HadoopJobExecHelper; import org.apache.hadoop.hive.ql.exec.tez.TezJobExecHelper; import org.apache.hadoop.hive.ql.parse.HiveParser; @@ -474,59 +472,61 @@ public void processSelectDatabase(CliSessionState ss) throws IOException { } } - public static Completor[] getCommandCompletor () { - // SimpleCompletor matches against a pre-defined wordlist + public static Completer[] getCommandCompleter() { + // StringsCompleter matches against a pre-defined wordlist // We start with an empty wordlist and build it up - SimpleCompletor sc = new SimpleCompletor(new String[0]); + List candidateStrings = new ArrayList(); // We add Hive function names // For functions that aren't infix operators, we add an open // parenthesis at the end. for (String s : FunctionRegistry.getFunctionNames()) { if (s.matches("[a-z_]+")) { - sc.addCandidateString(s + "("); + candidateStrings.add(s + "("); } else { - sc.addCandidateString(s); + candidateStrings.add(s); } } // We add Hive keywords, including lower-cased versions for (String s : HiveParser.getKeywords()) { - sc.addCandidateString(s); - sc.addCandidateString(s.toLowerCase()); + candidateStrings.add(s); + candidateStrings.add(s.toLowerCase()); } + StringsCompleter strCompleter = new StringsCompleter(candidateStrings); + // Because we use parentheses in addition to whitespace // as a keyword delimiter, we need to define a new ArgumentDelimiter // that recognizes parenthesis as a delimiter. - ArgumentDelimiter delim = new AbstractArgumentDelimiter () { + ArgumentDelimiter delim = new AbstractArgumentDelimiter() { @Override - public boolean isDelimiterChar (String buffer, int pos) { + public boolean isDelimiterChar(CharSequence buffer, int pos) { char c = buffer.charAt(pos); return (Character.isWhitespace(c) || c == '(' || c == ')' || - c == '[' || c == ']'); + c == '[' || c == ']'); } }; // The ArgumentCompletor allows us to match multiple tokens // in the same line. - final ArgumentCompletor ac = new ArgumentCompletor(sc, delim); + final ArgumentCompleter argCompleter = new ArgumentCompleter(delim, strCompleter); // By default ArgumentCompletor is in "strict" mode meaning // a token is only auto-completed if all prior tokens // match. We don't want that since there are valid tokens // that are not in our wordlist (eg. table and column names) - ac.setStrict(false); + argCompleter.setStrict(false); // ArgumentCompletor always adds a space after a matched token. // This is undesirable for function names because a space after // the opening parenthesis is unnecessary (and uncommon) in Hive. // We stack a custom Completor on top of our ArgumentCompletor // to reverse this. - Completor completor = new Completor () { + Completer customCompletor = new Completer () { @Override public int complete (String buffer, int offset, List completions) { List comp = completions; - int ret = ac.complete(buffer, offset, completions); + int ret = argCompleter.complete(buffer, offset, completions); // ConsoleReader will do the substitution if and only if there // is exactly one valid completion, so we ignore other cases. if (completions.size() == 1) { @@ -543,16 +543,24 @@ public int complete (String buffer, int offset, List completions) { for (int i = 0; i < vars.length; i++) { vars[i] = confs[i].varname; } - SimpleCompletor conf = new SimpleCompletor(vars); - conf.setDelimiter("."); + ArgumentCompleter.ArgumentDelimiter delimiter = new ArgumentCompleter.AbstractArgumentDelimiter() { + @Override + public boolean isDelimiterChar(CharSequence buffer, int pos) { + char c = buffer.charAt(pos); + return c == '.'; + } + }; - SimpleCompletor set = new SimpleCompletor("set") { + StringsCompleter setCompleter = new StringsCompleter("setCompleter") { @Override public int complete(String buffer, int cursor, List clist) { - return buffer != null && buffer.equals("set") ? super.complete(buffer, cursor, clist) : -1; + return buffer != null && buffer.equals("setCompleter") ? super.complete(buffer, cursor, clist) : -1; } }; - ArgumentCompletor propCompletor = new ArgumentCompletor(new Completor[]{set, conf}) { + + ArgumentCompleter confCompleter = new ArgumentCompleter(delimiter, setCompleter); + + ArgumentCompleter propCompleter = new ArgumentCompleter(new Completer[]{setCompleter, confCompleter}) { @Override @SuppressWarnings("unchecked") public int complete(String buffer, int offset, List completions) { @@ -563,7 +571,7 @@ public int complete(String buffer, int offset, List completions) { return ret; } }; - return new Completor[] {propCompletor, completor}; + return new Completer[] {propCompleter, customCompletor}; } public static void main(String[] args) throws Exception { @@ -675,8 +683,8 @@ private int executeDriver(CliSessionState ss, HiveConf conf, OptionsProcessor o ConsoleReader reader = getConsoleReader(); reader.setBellEnabled(false); // reader.setDebug(new PrintWriter(new FileWriter("writer.debug", true))); - for (Completor completor : getCommandCompletor()) { - reader.addCompletor(completor); + for (Completer completer : getCommandCompleter()) { + reader.addCompleter(completer); } String line; @@ -685,7 +693,7 @@ private int executeDriver(CliSessionState ss, HiveConf conf, OptionsProcessor o try { if ((new File(historyDirectory)).exists()) { String historyFile = historyDirectory + File.separator + HISTORYFILE; - reader.setHistory(new History(new File(historyFile))); + reader.setHistory(new FileHistory(new File(historyFile))); } else { System.err.println("WARNING: Directory for Hive history file: " + historyDirectory + " does not exist. History will not be available during this session."); diff --git cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java index 88a37d5..55607f3 100644 --- cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java +++ cli/src/test/org/apache/hadoop/hive/cli/TestCliDriverMethods.java @@ -42,9 +42,9 @@ import java.util.List; import java.util.Map; -import jline.ArgumentCompletor; -import jline.Completor; -import jline.ConsoleReader; +import jline.console.ConsoleReader; +import jline.console.completer.ArgumentCompleter; +import jline.console.completer.Completer; import junit.framework.TestCase; import org.apache.commons.io.FileUtils; @@ -141,20 +141,20 @@ private PrintStream headerPrintingTestDriver(Schema mockSchema) throws CommandNe public void testGetCommandCompletor() { - Completor[] completors = CliDriver.getCommandCompletor(); + Completer[] completors = CliDriver.getCommandCompleter(); assertEquals(2, completors.length); - assertTrue(completors[0] instanceof ArgumentCompletor); - assertTrue(completors[1] instanceof Completor); + assertTrue(completors[0] instanceof ArgumentCompleter); + assertTrue(completors[1] instanceof Completer); //comletor add space after last delimeter - ListtestList=new ArrayList(Arrays.asList(new String[]{")"})); + ListtestList=new ArrayList(Arrays.asList(new String[]{")"})); completors[1].complete("fdsdfsdf", 0, testList); assertEquals(") ", testList.get(0)); - testList=new ArrayList(); + testList=new ArrayList(); completors[1].complete("len", 0, testList); - assertTrue(testList.get(0).endsWith("length(")); + assertTrue(testList.get(0).toString().endsWith("length(")); - testList=new ArrayList(); + testList=new ArrayList(); completors[0].complete("set f", 0, testList); assertEquals("set", testList.get(0)); diff --git hcatalog/hcatalog-pig-adapter/pom.xml hcatalog/hcatalog-pig-adapter/pom.xml index 2d959e6..53a55dc 100644 --- hcatalog/hcatalog-pig-adapter/pom.xml +++ hcatalog/hcatalog-pig-adapter/pom.xml @@ -54,6 +54,12 @@ test + jline + jline + 0.9.94 + test + + org.apache.hive hive-exec ${project.version} diff --git pom.xml pom.xml index ec8c4fe..ddcb971 100644 --- pom.xml +++ pom.xml @@ -128,7 +128,7 @@ 3.0.1 7.6.0.v20120127 1.14 - 0.9.94 + 2.12 1.11 1.1 3.5.2 diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java index dea3460..3de5556 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezJobMonitor.java @@ -62,7 +62,7 @@ import java.util.SortedSet; import java.util.TreeSet; -import jline.Terminal; +import jline.TerminalFactory; /** * TezJobMonitor keeps track of a tez job while it's being executed. It will @@ -232,7 +232,7 @@ public void repositionCursor() { * @return - width of terminal */ public int getTerminalWidth() { - return Terminal.getTerminal().getTerminalWidth(); + return TerminalFactory.get().getWidth(); } /**