diff --git a/LICENSE b/LICENSE index c973c36..96e0c40 100644 --- a/LICENSE +++ b/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 contains + defend, and hold each Contributor harmless for any liability incurred by, or claims asserted against, such Contributor by reason of your accepting any such warranty or additional liability. @@ -432,7 +432,7 @@ 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: +For the org.apache.hive.sqlline.ClassNameCompleter class: Copyright (c) 2002-2006, Marc Prud'hommeaux All rights reserved. @@ -465,4 +465,4 @@ 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 +OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/beeline/pom.xml b/beeline/pom.xml index dcedf74..0d293c1 100644 --- a/beeline/pom.xml +++ b/beeline/pom.xml @@ -32,7 +32,7 @@ - + org.apache.hive @@ -54,6 +54,11 @@ hive-jdbc ${project.version} + + org.apache.hive + hive-sqlline + ${project.version} + commons-cli @@ -76,20 +81,10 @@ ${commons-io.version} - jline - jline - ${jline.version} - - org.apache.thrift libthrift ${libthrift.version} - - net.sf.supercsv - super-csv - ${super-csv.version} - org.apache.hive diff --git a/beeline/src/java/org/apache/hive/beeline/AbstractCommandHandler.java b/beeline/src/java/org/apache/hive/beeline/AbstractCommandHandler.java deleted file mode 100644 index 625879b..0000000 --- a/beeline/src/java/org/apache/hive/beeline/AbstractCommandHandler.java +++ /dev/null @@ -1,110 +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.Arrays; -import java.util.LinkedList; -import java.util.List; - -import jline.console.completer.Completer; -import jline.console.completer.NullCompleter; - -/** - * An abstract implementation of CommandHandler. - * - */ -public abstract class AbstractCommandHandler implements CommandHandler { - private final BeeLine beeLine; - private final String name; - private final String[] names; - private final String helpText; - private Completer[] parameterCompleters = new Completer[0]; - - protected transient Throwable lastException; - - public AbstractCommandHandler(BeeLine beeLine, String[] names, String helpText, - Completer[] completors) { - this.beeLine = beeLine; - name = names[0]; - this.names = names; - this.helpText = helpText; - if (completors == null || completors.length == 0) { - parameterCompleters = new Completer[] { new NullCompleter() }; - } else { - List c = new LinkedList(Arrays.asList(completors)); - c.add(new NullCompleter()); - parameterCompleters = c.toArray(new Completer[0]); - } - } - - @Override - public String getHelpText() { - return helpText; - } - - - @Override - public String getName() { - return name; - } - - - @Override - public String[] getNames() { - return names; - } - - - @Override - public String matches(String line) { - if (line == null || line.length() == 0) { - return null; - } - - String[] parts = beeLine.split(line); - if (parts == null || parts.length == 0) { - return null; - } - - for (String name2 : names) { - if (name2.startsWith(parts[0])) { - return name2; - } - } - return null; - } - - public void setParameterCompleters(Completer[] parameterCompleters) { - this.parameterCompleters = parameterCompleters; - } - - @Override - public Completer[] getParameterCompleters() { - return parameterCompleters; - } - - @Override - public Throwable getLastException() { - return lastException; - } -} diff --git a/beeline/src/java/org/apache/hive/beeline/AbstractOutputFormat.java b/beeline/src/java/org/apache/hive/beeline/AbstractOutputFormat.java deleted file mode 100644 index 97d27eb..0000000 --- a/beeline/src/java/org/apache/hive/beeline/AbstractOutputFormat.java +++ /dev/null @@ -1,50 +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; - -/** - * Abstract OutputFormat. - * - */ -abstract class AbstractOutputFormat implements OutputFormat { - - public int print(Rows rows) { - int count = 0; - Rows.Row header = (Rows.Row) rows.next(); - printHeader(header); - - while (rows.hasNext()) { - printRow(rows, header, (Rows.Row) rows.next()); - count++; - } - printFooter(header); - return count; - } - - - abstract void printHeader(Rows.Row header); - - abstract void printFooter(Rows.Row header); - - abstract void printRow(Rows rows, Rows.Row header, Rows.Row row); -} \ No newline at end of file diff --git a/beeline/src/java/org/apache/hive/beeline/BeeLine.java b/beeline/src/java/org/apache/hive/beeline/BeeLine.java index 630ead4..3278247 100644 --- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java +++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java @@ -15,636 +15,205 @@ * 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.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.Closeable; -import java.io.EOFException; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileWriter; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.OptionBuilder; +import org.apache.commons.cli.Options; + +import org.apache.hive.jdbc.HiveStatement; +import org.apache.hive.sqlline.SqlLine; +import org.apache.hive.sqlline.SqlLineOpts; + import java.io.IOException; import java.io.InputStream; -import java.io.PrintStream; -import java.io.PrintWriter; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.lang.reflect.Modifier; -import java.net.JarURLConnection; -import java.net.URL; -import java.net.URLConnection; -import java.sql.Connection; -import java.sql.DatabaseMetaData; -import java.sql.Driver; -import java.sql.DriverManager; -import java.sql.ResultSet; import java.sql.SQLException; -import java.sql.SQLWarning; import java.sql.Statement; -import java.text.ChoiceFormat; -import java.text.MessageFormat; import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.Date; import java.util.Enumeration; -import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.ListIterator; import java.util.Map; +import java.util.MissingResourceException; import java.util.Properties; import java.util.ResourceBundle; import java.util.Set; import java.util.SortedSet; -import java.util.StringTokenizer; -import java.util.TreeMap; import java.util.TreeSet; -import java.util.jar.Attributes; -import java.util.jar.Manifest; -import java.util.zip.ZipEntry; -import java.util.zip.ZipFile; - -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; -import org.apache.commons.cli.Options; -import org.apache.commons.cli.ParseException; -import org.apache.hadoop.io.IOUtils; - /** - * A console SQL shell with command completion. - *

- * TODO: - *

    - *
  • User-friendly connection prompts
  • - *
  • Page results
  • - *
  • Handle binary data (blob fields)
  • - *
  • Implement command aliases
  • - *
  • Stored procedure execution
  • - *
  • Binding parameters to prepared statements
  • - *
  • Scripting language
  • - *
  • XA transactions
  • - *
- * + * A console SQL shell for Hive, with command completion. */ -public class BeeLine implements Closeable { - private static final ResourceBundle resourceBundle = - ResourceBundle.getBundle(BeeLine.class.getSimpleName()); - private final BeeLineSignalHandler signalHandler = null; - private static final String separator = System.getProperty("line.separator"); - private boolean exit = false; - private final DatabaseConnections connections = new DatabaseConnections(); - public static final String COMMAND_PREFIX = "!"; - private final Completer beeLineCommandCompleter; - private Collection drivers = null; - private final BeeLineOpts opts = new BeeLineOpts(this, System.getProperties()); - private String lastProgress = null; - private final Map seenWarnings = new HashMap(); - private final Commands commands = new Commands(this); - private OutputFile scriptOutputFile = null; - private OutputFile recordOutputFile = null; - private PrintStream outputStream = new PrintStream(System.out, true); - private PrintStream errorStream = new PrintStream(System.err, true); - private ConsoleReader consoleReader; - private List batch = null; - private final Reflector reflector; - - private History history; - - private static final Options options = new Options(); +public class BeeLine extends SqlLine { + public static final int DEFAULT_MAX_WIDTH = 80; + public static final int DEFAULT_MAX_HEIGHT = 80; - public static final String BEELINE_DEFAULT_JDBC_DRIVER = "org.apache.hive.jdbc.HiveDriver"; + public static final String BEELINE_DEFAULT_JDBC_DRIVER = + "org.apache.hive.jdbc.HiveDriver"; public static final String BEELINE_DEFAULT_JDBC_URL = "jdbc:hive2://"; private static final String SCRIPT_OUTPUT_PREFIX = ">>>"; private static final int SCRIPT_OUTPUT_PAD_SIZE = 5; - private static final int ERRNO_OK = 0; - private static final int ERRNO_ARGS = 1; - private static final int ERRNO_OTHER = 2; - - private static final String HIVE_VAR_PREFIX = "--hivevar"; - private static final String HIVE_CONF_PREFIX = "--hiveconf"; - - private final Map formats = map(new Object[] { - "vertical", new VerticalOutputFormat(this), - "table", new TableOutputFormat(this), - "csv2", new SeparatedValuesOutputFormat(this, ','), - "tsv2", new SeparatedValuesOutputFormat(this, '\t'), - "dsv", new SeparatedValuesOutputFormat(this, BeeLineOpts.DEFAULT_DELIMITER_FOR_DSV), - "csv", new DeprecatedSeparatedValuesOutputFormat(this, ','), - "tsv", new DeprecatedSeparatedValuesOutputFormat(this, '\t'), - "xmlattr", new XMLAttributeOutputFormat(this), - "xmlelements", new XMLElementOutputFormat(this), - }); - - - final CommandHandler[] commandHandlers = new CommandHandler[] { - new ReflectiveCommandHandler(this, new String[] {"quit", "done", "exit"}, - null), - new ReflectiveCommandHandler(this, new String[] {"connect", "open"}, - new Completer[] {new StringsCompleter(getConnectionURLExamples())}), - new ReflectiveCommandHandler(this, new String[] {"describe"}, - new Completer[] {new TableNameCompletor(this)}), - new ReflectiveCommandHandler(this, new String[] {"indexes"}, - new Completer[] {new TableNameCompletor(this)}), - new ReflectiveCommandHandler(this, new String[] {"primarykeys"}, - new Completer[] {new TableNameCompletor(this)}), - new ReflectiveCommandHandler(this, new String[] {"exportedkeys"}, - new Completer[] {new TableNameCompletor(this)}), - new ReflectiveCommandHandler(this, new String[] {"manual"}, - null), - new ReflectiveCommandHandler(this, new String[] {"importedkeys"}, - new Completer[] {new TableNameCompletor(this)}), - new ReflectiveCommandHandler(this, new String[] {"procedures"}, - null), - new ReflectiveCommandHandler(this, new String[] {"tables"}, - null), - new ReflectiveCommandHandler(this, new String[] {"typeinfo"}, - null), - new ReflectiveCommandHandler(this, new String[] {"columns"}, - new Completer[] {new TableNameCompletor(this)}), - new ReflectiveCommandHandler(this, new String[] {"reconnect"}, - null), - new ReflectiveCommandHandler(this, new String[] {"dropall"}, - new Completer[] {new TableNameCompletor(this)}), - new ReflectiveCommandHandler(this, new String[] {"history"}, - null), - new ReflectiveCommandHandler(this, new String[] {"metadata"}, - new Completer[] { - new StringsCompleter(getMetadataMethodNames())}), - new ReflectiveCommandHandler(this, new String[] {"nativesql"}, - null), - new ReflectiveCommandHandler(this, new String[] {"dbinfo"}, - null), - new ReflectiveCommandHandler(this, new String[] {"rehash"}, - null), - new ReflectiveCommandHandler(this, new String[] {"verbose"}, - null), - new ReflectiveCommandHandler(this, new String[] {"run"}, - new Completer[] {new FileNameCompleter()}), - new ReflectiveCommandHandler(this, new String[] {"batch"}, - null), - new ReflectiveCommandHandler(this, new String[] {"list"}, - null), - new ReflectiveCommandHandler(this, new String[] {"all"}, - null), - new ReflectiveCommandHandler(this, new String[] {"go", "#"}, - null), - new ReflectiveCommandHandler(this, new String[] {"script"}, - new Completer[] {new FileNameCompleter()}), - new ReflectiveCommandHandler(this, new String[] {"record"}, - new Completer[] {new FileNameCompleter()}), - new ReflectiveCommandHandler(this, new String[] {"brief"}, - null), - new ReflectiveCommandHandler(this, new String[] {"close"}, - null), - new ReflectiveCommandHandler(this, new String[] {"closeall"}, - null), - new ReflectiveCommandHandler(this, new String[] {"isolation"}, - new Completer[] {new StringsCompleter(getIsolationLevels())}), - new ReflectiveCommandHandler(this, new String[] {"outputformat"}, - 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 Completer[] {new FileNameCompleter()}), - new ReflectiveCommandHandler(this, new String[] {"rollback"}, - null), - new ReflectiveCommandHandler(this, new String[] {"help", "?"}, - null), - new ReflectiveCommandHandler(this, new String[] {"set"}, - getOpts().optionCompleters()), - new ReflectiveCommandHandler(this, new String[] {"save"}, - null), - new ReflectiveCommandHandler(this, new String[] {"scan"}, - null), - new ReflectiveCommandHandler(this, new String[] {"sql"}, - null), - new ReflectiveCommandHandler(this, new String[] {"sh"}, - null), - new ReflectiveCommandHandler(this, new String[] {"call"}, - null), - new ReflectiveCommandHandler(this, new String[] {"nullemptystring"}, - new Completer[] {new BooleanCompleter()}), - }; - - - static final SortedSet KNOWN_DRIVERS = new TreeSet(Arrays.asList( - new String[] { + static final SortedSet HIVE_DRIVERS = + new TreeSet(Arrays.asList( "org.apache.hive.jdbc.HiveDriver", - "org.apache.hadoop.hive.jdbc.HiveDriver", - })); - - - static { - try { - Class.forName("jline.console.ConsoleReader"); - } catch (Throwable t) { - throw new ExceptionInInitializerError("jline-missing"); - } + "com.mysql.jdbc.DatabaseMetaData")); + + /** Prefix to the name of properties (passed to getConnection) that specify + * Hive variables. */ + private static final String HIVE_VAR_PREFIX = "hivevar:"; + + /** Prefix to the name of properties (passed to getConnection) that specify + * Hive configuration variables. */ + private static final String HIVE_CONF_PREFIX = "hiveconf:"; + + /** Combined resource bundle that first looks in BeeLine.properties, then in + * SqlLine.properties. */ + private static final ResourceBundle BEE_LINE_RESOURCE_BUNDLE = + new ChainResourceBundle( + Arrays.asList(ResourceBundle.getBundle(BeeLine.class.getName()), + SqlLine.RESOURCE_BUNDLE)); + + private static final Options COMMAND_LINE_OPTIONS = createBeeLineOptions(); + + @SuppressWarnings("AccessStaticViaInstance") + protected static Options createBeeLineOptions() { + // BeeLine options are an extension to SqlLine. + final Options options = createSqlLineOptions(); + + // Synchronized prevents other threads building options using the same + // static workspace in OptionBuilder. + synchronized (OptionBuilder.class) { + // -a + options.addOption(OptionBuilder + .hasArg() + .withArgName("authType") + .withDescription("the authentication type") + .create('a')); + + // Substitution option --hivevar + options.addOption(OptionBuilder + .withValueSeparator() + .hasArgs(2) + .withArgName("key=value") + .withLongOpt("hivevar") + .withDescription("hive variable name and value") + .create()); + + //hive conf option --hiveconf + options.addOption(OptionBuilder + .withValueSeparator() + .hasArgs(2) + .withArgName("property=value") + .withLongOpt("hiveconf") + .withDescription("Use value for given property") + .create()); + } + return options; } - static { - // -d - options.addOption(OptionBuilder - .hasArg() - .withArgName("driver class") - .withDescription("the driver class to use") - .create('d')); - - // -u - options.addOption(OptionBuilder - .hasArg() - .withArgName("database url") - .withDescription("the JDBC URL to connect to") - .create('u')); - - // -n - options.addOption(OptionBuilder - .hasArg() - .withArgName("username") - .withDescription("the username to connect as") - .create('n')); - - // -p - options.addOption(OptionBuilder - .hasArg() - .withArgName("password") - .withDescription("the password to connect as") - .create('p')); - - // -a - options.addOption(OptionBuilder - .hasArg() - .withArgName("authType") - .withDescription("the authentication type") - .create('a')); - - // -i - options.addOption(OptionBuilder - .hasArg() - .withArgName("init") - .withDescription("script file for initialization") - .create('i')); - - // -e - options.addOption(OptionBuilder - .hasArgs() - .withArgName("query") - .withDescription("query that should be executed") - .create('e')); - - // -f