diff --git beeline/build.xml beeline/build.xml
new file mode 100644
index 0000000..d583915
--- /dev/null
+++ beeline/build.xml
@@ -0,0 +1,50 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git beeline/ivy.xml beeline/ivy.xml
new file mode 100644
index 0000000..cfefa76
--- /dev/null
+++ beeline/ivy.xml
@@ -0,0 +1,57 @@
+
+
+
+
+
+
+ The Apache Hive (TM) data warehouse software facilitates querying and managing large datasets residing in distributed storage.
+ https://cwiki.apache.org/confluence/display/Hive/Home
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git beeline/src/java/org/apache/hive/beeline/AbstractCommandHandler.java beeline/src/java/org/apache/hive/beeline/AbstractCommandHandler.java
new file mode 100644
index 0000000..30f43c4
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/AbstractCommandHandler.java
@@ -0,0 +1,126 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+
+import jline.Completor;
+import jline.NullCompletor;
+
+/**
+ * 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 Completor[] parameterCompletors = new Completor[0];
+
+
+ public AbstractCommandHandler(BeeLine beeLine, String[] names, String helpText,
+ Completor[] completors) {
+ this.beeLine = beeLine;
+ name = names[0];
+ this.names = names;
+ this.helpText = helpText;
+ if (completors == null || completors.length == 0) {
+ parameterCompletors = new Completor[] { new NullCompletor() };
+ } else {
+ List c = new LinkedList(Arrays.asList(completors));
+ c.add(new NullCompletor());
+ parameterCompletors = c.toArray(new Completor[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 setParameterCompletors(Completor[] parameterCompletors) {
+ this.parameterCompletors = parameterCompletors;
+ }
+
+ @Override
+ public Completor[] getParameterCompletors() {
+ return parameterCompletors;
+ }
+}
diff --git beeline/src/java/org/apache/hive/beeline/AbstractOutputFormat.java beeline/src/java/org/apache/hive/beeline/AbstractOutputFormat.java
new file mode 100644
index 0000000..cee9713
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/AbstractOutputFormat.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+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 beeline/src/java/org/apache/hive/beeline/BeeLine.java beeline/src/java/org/apache/hive/beeline/BeeLine.java
new file mode 100644
index 0000000..1d822ba
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/BeeLine.java
@@ -0,0 +1,1700 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+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.LinkedList;
+import java.util.List;
+import java.util.Map;
+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.ClassNameCompletor;
+import jline.Completor;
+import jline.ConsoleReader;
+import jline.FileNameCompletor;
+import jline.SimpleCompletor;
+
+
+/**
+ * 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
+ *
+ *
+ */
+public class BeeLine {
+ private static final ResourceBundle resourceBundle =
+ ResourceBundle.getBundle(BeeLine.class.getName());
+ private 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 Completor beeLineCommandCompletor;
+ 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;
+
+ 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 final Map formats = map(new Object[] {
+ "vertical", new VerticalOutputFormat(this),
+ "table", new TableOutputFormat(this),
+ "csv", new SeparatedValuesOutputFormat(this, ','),
+ "tsv", new SeparatedValuesOutputFormat(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 Completor[] {new SimpleCompletor(getConnectionURLExamples())}),
+ new ReflectiveCommandHandler(this, new String[] {"describe"},
+ new Completor[] {new TableNameCompletor(this)}),
+ new ReflectiveCommandHandler(this, new String[] {"indexes"},
+ new Completor[] {new TableNameCompletor(this)}),
+ new ReflectiveCommandHandler(this, new String[] {"primarykeys"},
+ new Completor[] {new TableNameCompletor(this)}),
+ new ReflectiveCommandHandler(this, new String[] {"exportedkeys"},
+ new Completor[] {new TableNameCompletor(this)}),
+ new ReflectiveCommandHandler(this, new String[] {"manual"},
+ null),
+ new ReflectiveCommandHandler(this, new String[] {"importedkeys"},
+ new Completor[] {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 Completor[] {new TableNameCompletor(this)}),
+ new ReflectiveCommandHandler(this, new String[] {"reconnect"},
+ null),
+ new ReflectiveCommandHandler(this, new String[] {"dropall"},
+ new Completor[] {new TableNameCompletor(this)}),
+ new ReflectiveCommandHandler(this, new String[] {"history"},
+ null),
+ new ReflectiveCommandHandler(this, new String[] {"metadata"},
+ new Completor[] {
+ new SimpleCompletor(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 Completor[] {new FileNameCompletor()}),
+ 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 Completor[] {new FileNameCompletor()}),
+ new ReflectiveCommandHandler(this, new String[] {"record"},
+ new Completor[] {new FileNameCompletor()}),
+ 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 Completor[] {new SimpleCompletor(getIsolationLevels())}),
+ new ReflectiveCommandHandler(this, new String[] {"outputformat"},
+ new Completor[] {new SimpleCompletor(
+ 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 ReflectiveCommandHandler(this, new String[] {"rollback"},
+ null),
+ new ReflectiveCommandHandler(this, new String[] {"help", "?"},
+ null),
+ new ReflectiveCommandHandler(this, new String[] {"set"},
+ getOpts().optionCompletors()),
+ 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[] {"call"},
+ null),
+ };
+
+
+ static final SortedSet KNOWN_DRIVERS = new TreeSet(Arrays.asList(
+ new String[] {
+ "org.apache.hive.jdbc.HiveDriver",
+ "org.apache.hadoop.hive.jdbc.HiveDriver",
+ }));
+
+
+ static {
+ try {
+ Class.forName("jline.ConsoleReader");
+ } catch (Throwable t) {
+ throw new ExceptionInInitializerError("jline-missing");
+ }
+ }
+
+
+ static Manifest getManifest() throws IOException {
+ URL base = BeeLine.class.getResource("/META-INF/MANIFEST.MF");
+ URLConnection c = base.openConnection();
+ if (c instanceof JarURLConnection) {
+ return ((JarURLConnection) c).getManifest();
+ }
+ return null;
+ }
+
+
+ String getManifestAttribute(String name) {
+ try {
+ Manifest m = getManifest();
+ if (m == null) {
+ return "??";
+ }
+
+ Attributes attrs = m.getAttributes("beeline");
+ if (attrs == null) {
+ return "???";
+ }
+
+ String val = attrs.getValue(name);
+ if (val == null || "".equals(val)) {
+ return "????";
+ }
+
+ return val;
+ } catch (Exception e) {
+ e.printStackTrace(errorStream);
+ return "?????";
+ }
+ }
+
+
+ String getApplicationTitle() {
+ Package pack = BeeLine.class.getPackage();
+
+ return loc("app-introduction", new Object[] {
+ "Beeline",
+ pack.getImplementationVersion() == null ? "???"
+ : pack.getImplementationVersion(),
+ "Apache Hive",
+ // getManifestAttribute ("Specification-Title"),
+ // getManifestAttribute ("Implementation-Version"),
+ // getManifestAttribute ("Implementation-ReleaseDate"),
+ // getManifestAttribute ("Implementation-Vendor"),
+ // getManifestAttribute ("Implementation-License"),
+ });
+ }
+
+ String getApplicationContactInformation() {
+ return getManifestAttribute("Implementation-Vendor");
+ }
+
+ String loc(String res) {
+ return loc(res, new Object[0]);
+ }
+
+ String loc(String res, int param) {
+ try {
+ return MessageFormat.format(
+ new ChoiceFormat(resourceBundle.getString(res)).format(param),
+ new Object[] {new Integer(param)});
+ } catch (Exception e) {
+ return res + ": " + param;
+ }
+ }
+
+ String loc(String res, Object param1) {
+ return loc(res, new Object[] {param1});
+ }
+
+ String loc(String res, Object param1, Object param2) {
+ return loc(res, new Object[] {param1, param2});
+ }
+
+ String loc(String res, Object[] params) {
+ try {
+ return MessageFormat.format(resourceBundle.getString(res), params);
+ } catch (Exception e) {
+ e.printStackTrace(getErrorStream());
+ try {
+ return res + ": " + Arrays.asList(params);
+ } catch (Exception e2) {
+ return res;
+ }
+ }
+ }
+
+ protected String locElapsedTime(long milliseconds) {
+ if (getOpts().getShowElapsedTime()) {
+ return loc("time-ms", new Object[] {new Double(milliseconds / 1000d)});
+ }
+ return "";
+ }
+
+
+ /**
+ * Starts the program.
+ */
+ public static void main(String[] args) throws IOException {
+ mainWithInputRedirection(args, null);
+ }
+
+ /**
+ * Starts the program with redirected input. For redirected output,
+ * setOutputStream() and setErrorStream can be used.
+ *
+ * @param args
+ * same as main()
+ *
+ * @param inputStream
+ * redirected input, or null to use standard input
+ */
+ public static void mainWithInputRedirection(String[] args, InputStream inputStream)
+ throws IOException {
+ BeeLine beeLine = new BeeLine();
+ beeLine.begin(args, inputStream);
+
+ // exit the system: useful for Hypersonic and other
+ // badly-behaving systems
+ if (!Boolean.getBoolean(BeeLineOpts.PROPERTY_NAME_EXIT)) {
+ System.exit(0);
+ }
+ }
+
+
+ public BeeLine() {
+ beeLineCommandCompletor = new BeeLineCommandCompletor(this);
+ reflector = new Reflector(this);
+
+ // attempt to dynamically load signal handler
+ /* TODO disable signal handler
+ try {
+ Class> handlerClass =
+ Class.forName("org.apache.hive.beeline.SunSignalHandler");
+ signalHandler = (BeeLineSignalHandler)
+ handlerClass.newInstance();
+ } catch (Throwable t) {
+ // ignore and leave cancel functionality disabled
+ }
+ */
+ }
+
+
+ DatabaseConnection getDatabaseConnection() {
+ return getDatabaseConnections().current();
+ }
+
+
+ Connection getConnection() throws SQLException {
+ if (getDatabaseConnections().current() == null) {
+ throw new IllegalArgumentException(loc("no-current-connection"));
+ }
+ if (getDatabaseConnections().current().getConnection() == null) {
+ throw new IllegalArgumentException(loc("no-current-connection"));
+ }
+ return getDatabaseConnections().current().getConnection();
+ }
+
+
+ DatabaseMetaData getDatabaseMetaData() {
+ if (getDatabaseConnections().current() == null) {
+ throw new IllegalArgumentException(loc("no-current-connection"));
+ }
+ if (getDatabaseConnections().current().getDatabaseMetaData() == null) {
+ throw new IllegalArgumentException(loc("no-current-connection"));
+ }
+ return getDatabaseConnections().current().getDatabaseMetaData();
+ }
+
+
+ public String[] getIsolationLevels() {
+ return new String[] {
+ "TRANSACTION_NONE",
+ "TRANSACTION_READ_COMMITTED",
+ "TRANSACTION_READ_UNCOMMITTED",
+ "TRANSACTION_REPEATABLE_READ",
+ "TRANSACTION_SERIALIZABLE",
+ };
+ }
+
+
+ public String[] getMetadataMethodNames() {
+ try {
+ TreeSet mnames = new TreeSet();
+ Method[] m = DatabaseMetaData.class.getDeclaredMethods();
+ for (int i = 0; m != null && i < m.length; i++) {
+ mnames.add(m[i].getName());
+ }
+ return mnames.toArray(new String[0]);
+ } catch (Throwable t) {
+ return new String[0];
+ }
+ }
+
+
+ public String[] getConnectionURLExamples() {
+ return new String[] {
+ "jdbc:JSQLConnect:///database=",
+ "jdbc:cloudscape:;create=true",
+ "jdbc:twtds:sqlserver:///",
+ "jdbc:daffodilDB_embedded:;create=true",
+ "jdbc:datadirect:db2://:50000;databaseName=",
+ "jdbc:inetdae::1433",
+ "jdbc:datadirect:oracle://:1521;SID=;MaxPooledStatements=0",
+ "jdbc:datadirect:sqlserver://:1433;SelectMethod=cursor;DatabaseName=",
+ "jdbc:datadirect:sybase://:5000",
+ "jdbc:db2:///",
+ "jdbc:hive2://",
+ "jdbc:hsqldb:",
+ "jdbc:idb:.properties",
+ "jdbc:informix-sqli://:1526/:INFORMIXSERVER=",
+ "jdbc:interbase:////.gdb",
+ "jdbc:microsoft:sqlserver://:1433;DatabaseName=;SelectMethod=cursor",
+ "jdbc:mysql:///?autoReconnect=true",
+ "jdbc:oracle:thin:@:1521:",
+ "jdbc:pointbase:,database.home=,create=true",
+ "jdbc:postgresql://:5432/",
+ "jdbc:postgresql:net///",
+ "jdbc:sybase:Tds::4100/?ServiceName=",
+ "jdbc:weblogic:mssqlserver4:@:1433",
+ "jdbc:odbc:",
+ "jdbc:sequelink://:4003/[Oracle]",
+ "jdbc:sequelink://:4004/[Informix];Database=",
+ "jdbc:sequelink://:4005/[Sybase];Database=",
+ "jdbc:sequelink://:4006/[SQLServer];Database=",
+ "jdbc:sequelink://:4011/[ODBC MS Access];Database=",
+ "jdbc:openlink:///DSN=SQLServerDB/UID=sa/PWD=",
+ "jdbc:solid://://",
+ "jdbc:dbaw://:8889/",
+ };
+ }
+
+ /**
+ * Entry point to creating a {@link ColorBuffer} with color
+ * enabled or disabled depending on the value of {@link BeeLineOpts#getColor}.
+ */
+ ColorBuffer getColorBuffer() {
+ return new ColorBuffer(getOpts().getColor());
+ }
+
+
+ /**
+ * Entry point to creating a {@link ColorBuffer} with color
+ * enabled or disabled depending on the value of {@link BeeLineOpts#getColor}.
+ */
+ ColorBuffer getColorBuffer(String msg) {
+ return new ColorBuffer(msg, getOpts().getColor());
+ }
+
+
+ boolean initArgs(String[] args) {
+ List commands = new LinkedList();
+ List files = new LinkedList();
+ String driver = null, user = null, pass = null, url = null, cmd = null;
+
+ for (int i = 0; i < args.length; i++) {
+ if (args[i].equals("--help") || args[i].equals("-h")) {
+ usage();
+ return false;
+ }
+
+ // -- arguments are treated as properties
+ if (args[i].startsWith("--")) {
+ String[] parts = split(args[i].substring(2), "=");
+ debug(loc("setting-prop", Arrays.asList(parts)));
+ if (parts.length > 0) {
+ boolean ret;
+
+ if (parts.length >= 2) {
+ ret = getOpts().set(parts[0], parts[1], true);
+ } else {
+ ret = getOpts().set(parts[0], "true", true);
+ }
+
+ if (!ret) {
+ return false;
+ }
+
+ }
+ continue;
+ }
+
+ if (args[i].equals("-d")) {
+ driver = args[i++ + 1];
+ } else if (args[i].equals("-n")) {
+ user = args[i++ + 1];
+ } else if (args[i].equals("-p")) {
+ pass = args[i++ + 1];
+ } else if (args[i].equals("-u")) {
+ url = args[i++ + 1];
+ } else if (args[i].equals("-e")) {
+ commands.add(args[i++ + 1]);
+ } else {
+ files.add(args[i]);
+ }
+ }
+
+ // TODO: temporary disable this for easier debugging
+ /*
+ if (url == null) {
+ url = BEELINE_DEFAULT_JDBC_URL;
+ }
+ if (driver == null) {
+ driver = BEELINE_DEFAULT_JDBC_DRIVER;
+ }
+ */
+
+ if (url != null) {
+ String com = "!connect "
+ + url + " "
+ + (user == null || user.length() == 0 ? "''" : user) + " "
+ + (pass == null || pass.length() == 0 ? "''" : pass) + " "
+ + (driver == null ? "" : driver);
+ debug("issuing: " + com);
+ dispatch(com);
+ }
+
+ // now load properties files
+ for (Iterator i = files.iterator(); i.hasNext();) {
+ dispatch("!properties " + i.next());
+ }
+
+
+ if (commands.size() > 0) {
+ // for single command execute, disable color
+ getOpts().setColor(false);
+ getOpts().setHeaderInterval(-1);
+
+ for (Iterator i = commands.iterator(); i.hasNext();) {
+ String command = i.next().toString();
+ debug(loc("executing-command", command));
+ dispatch(command);
+ }
+ exit = true; // execute and exit
+ }
+ return true;
+ }
+
+
+ /**
+ * Start accepting input from stdin, and dispatch it
+ * to the appropriate {@link CommandHandler} until the
+ * global variable exit is true.
+ */
+ void begin(String[] args, InputStream inputStream) throws IOException {
+ try {
+ // load the options first, so we can override on the command line
+ getOpts().load();
+ } catch (Exception e) {
+ // nothing
+ }
+
+ ConsoleReader reader = getConsoleReader(inputStream);
+ if (!(initArgs(args))) {
+ usage();
+ return;
+ }
+
+ try {
+ info(getApplicationTitle());
+ } catch (Exception e) {
+ // ignore
+ }
+
+ while (!exit) {
+ try {
+ dispatch(reader.readLine(getPrompt()));
+ } catch (EOFException eof) {
+ // CTRL-D
+ commands.quit(null);
+ } catch (Throwable t) {
+ handleException(t);
+ }
+ }
+ // ### NOTE jvs 10-Aug-2004: Clean up any outstanding
+ // connections automatically.
+ commands.closeall(null);
+ }
+
+ public void close() {
+ commands.quit(null);
+ commands.closeall(null);
+ }
+
+ public ConsoleReader getConsoleReader(InputStream inputStream) throws IOException {
+ if (inputStream != null) {
+ // ### NOTE: fix for sf.net bug 879425.
+ consoleReader = new ConsoleReader(inputStream, new PrintWriter(System.out, true));
+ } else {
+ consoleReader = new ConsoleReader();
+ }
+
+ // setup history
+ ByteArrayInputStream historyBuffer = null;
+
+ if (new File(getOpts().getHistoryFile()).isFile()) {
+ try {
+ // save the current contents of the history buffer. This gets
+ // around a bug in JLine where setting the output before the
+ // input will clobber the history input, but setting the
+ // 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();
+ int n;
+ while ((n = historyIn.read()) != -1) {
+ hist.write(n);
+ }
+ historyIn.close();
+ historyBuffer = new ByteArrayInputStream(hist.toByteArray());
+ } catch (Exception e) {
+ handleException(e);
+ }
+ }
+
+ try {
+ // now set the output for the history
+ PrintWriter historyOut = new PrintWriter(new FileWriter(getOpts().getHistoryFile()), true);
+ consoleReader.getHistory().setOutput(historyOut);
+ } catch (Exception e) {
+ handleException(e);
+ }
+
+ try {
+ // now load in the previous history
+ if (historyBuffer != null) {
+ consoleReader.getHistory().load(historyBuffer);
+ }
+ } catch (Exception e) {
+ handleException(e);
+ }
+ consoleReader.addCompletor(new BeeLineCompletor(this));
+ return consoleReader;
+ }
+
+
+ void usage() {
+ output(loc("cmd-usage"));
+ }
+
+
+ /**
+ * Dispatch the specified line to the appropriate {@link CommandHandler}.
+ *
+ * @param line
+ * the commmand-line to dispatch
+ * @return true if the command was "successful"
+ */
+ boolean dispatch(String line) {
+ if (line == null) {
+ // exit
+ exit = true;
+ return true;
+ }
+
+ if (line.trim().length() == 0) {
+ return true;
+ }
+
+ if (isComment(line)) {
+ return true;
+ }
+
+ line = line.trim();
+
+ // save it to the current script, if any
+ if (scriptOutputFile != null) {
+ scriptOutputFile.addLine(line);
+ }
+
+ if (isHelpRequest(line)) {
+ line = "!help";
+ }
+
+ if (line.startsWith(COMMAND_PREFIX)) {
+ Map cmdMap = new TreeMap();
+ line = line.substring(1);
+ for (int i = 0; i < commandHandlers.length; i++) {
+ String match = commandHandlers[i].matches(line);
+ if (match != null) {
+ cmdMap.put(match, commandHandlers[i]);
+ }
+ }
+
+ if (cmdMap.size() == 0) {
+ return error(loc("unknown-command", line));
+ } else if (cmdMap.size() > 1) {
+ return error(loc("multiple-matches",
+ cmdMap.keySet().toString()));
+ } else {
+ return cmdMap.values().iterator().next()
+ .execute(line);
+ }
+ } else {
+ return commands.sql(line);
+ }
+ }
+
+ /**
+ * Test whether a line requires a continuation.
+ *
+ * @param line
+ * the line to be tested
+ *
+ * @return true if continuation required
+ */
+ boolean needsContinuation(String line) {
+ if (isHelpRequest(line)) {
+ return false;
+ }
+
+ if (line.startsWith(COMMAND_PREFIX)) {
+ return false;
+ }
+
+ if (isComment(line)) {
+ return false;
+ }
+
+ String trimmed = line.trim();
+
+ if (trimmed.length() == 0) {
+ return false;
+ }
+ return !trimmed.endsWith(";");
+ }
+
+ /**
+ * Test whether a line is a help request other than !help.
+ *
+ * @param line
+ * the line to be tested
+ *
+ * @return true if a help request
+ */
+ boolean isHelpRequest(String line) {
+ return line.equals("?") || line.equalsIgnoreCase("help");
+ }
+
+ /**
+ * Test whether a line is a comment.
+ *
+ * @param line
+ * the line to be tested
+ *
+ * @return true if a comment
+ */
+ boolean isComment(String line) {
+ // SQL92 comment prefix is "--"
+ // beeline also supports shell-style "#" prefix
+ return line.startsWith("#") || line.startsWith("--");
+ }
+
+ /**
+ * Print the specified message to the console
+ *
+ * @param msg
+ * the message to print
+ */
+ void output(String msg) {
+ output(msg, true);
+ }
+
+
+ void info(String msg) {
+ if (!(getOpts().isSilent())) {
+ output(msg, true, getErrorStream());
+ }
+ }
+
+
+ void info(ColorBuffer msg) {
+ if (!(getOpts().isSilent())) {
+ output(msg, true, getErrorStream());
+ }
+ }
+
+
+ /**
+ * Issue the specified error message
+ *
+ * @param msg
+ * the message to issue
+ * @return false always
+ */
+ boolean error(String msg) {
+ output(getColorBuffer().red(msg), true, getErrorStream());
+ return false;
+ }
+
+
+ boolean error(Throwable t) {
+ handleException(t);
+ return false;
+ }
+
+
+ void debug(String msg) {
+ if (getOpts().getVerbose()) {
+ output(getColorBuffer().blue(msg), true, getErrorStream());
+ }
+ }
+
+
+ void output(ColorBuffer msg) {
+ output(msg, true);
+ }
+
+
+ void output(String msg, boolean newline, PrintStream out) {
+ output(getColorBuffer(msg), newline, out);
+ }
+
+
+ void output(ColorBuffer msg, boolean newline) {
+ output(msg, newline, getOutputStream());
+ }
+
+
+ void output(ColorBuffer msg, boolean newline, PrintStream out) {
+ if (newline) {
+ out.println(msg.getColor());
+ } else {
+ out.print(msg.getColor());
+ }
+
+ if (recordOutputFile == null) {
+ return;
+ }
+
+ // only write to the record file if we are writing a line ...
+ // otherwise we might get garbage from backspaces and such.
+ if (newline) {
+ recordOutputFile.addLine(msg.getMono()); // always just write mono
+ } else {
+ recordOutputFile.print(msg.getMono());
+ }
+ }
+
+
+ /**
+ * Print the specified message to the console
+ *
+ * @param msg
+ * the message to print
+ * @param newline
+ * if false, do not append a newline
+ */
+ void output(String msg, boolean newline) {
+ output(getColorBuffer(msg), newline);
+ }
+
+
+ void autocommitStatus(Connection c) throws SQLException {
+ info(loc("autocommit-status", c.getAutoCommit() + ""));
+ }
+
+
+ /**
+ * Ensure that autocommit is on for the current connection
+ *
+ * @return true if autocommit is set
+ */
+ boolean assertAutoCommit() {
+ if (!(assertConnection())) {
+ return false;
+ }
+ try {
+ if (getDatabaseConnection().getConnection().getAutoCommit()) {
+ return error(loc("autocommit-needs-off"));
+ }
+ } catch (Exception e) {
+ return error(e);
+ }
+ return true;
+ }
+
+
+ /**
+ * Assert that we have an active, living connection. Print
+ * an error message if we do not.
+ *
+ * @return true if there is a current, active connection
+ */
+ boolean assertConnection() {
+ try {
+ if (getDatabaseConnection() == null || getDatabaseConnection().getConnection() == null) {
+ return error(loc("no-current-connection"));
+ }
+ if (getDatabaseConnection().getConnection().isClosed()) {
+ return error(loc("connection-is-closed"));
+ }
+ } catch (SQLException sqle) {
+ return error(loc("no-current-connection"));
+ }
+ return true;
+ }
+
+
+ /**
+ * Print out any warnings that exist for the current connection.
+ */
+ void showWarnings() {
+ try {
+ if (getDatabaseConnection().getConnection() == null
+ || !getOpts().getVerbose()) {
+ return;
+ }
+ showWarnings(getDatabaseConnection().getConnection().getWarnings());
+ } catch (Exception e) {
+ handleException(e);
+ }
+ }
+
+
+ /**
+ * Print the specified warning on the console, as well as
+ * any warnings that are returned from {@link SQLWarning#getNextWarning}.
+ *
+ * @param warn
+ * the {@link SQLWarning} to print
+ */
+ void showWarnings(SQLWarning warn) {
+ if (warn == null) {
+ return;
+ }
+
+ if (seenWarnings.get(warn) == null) {
+ // don't re-display warnings we have already seen
+ seenWarnings.put(warn, new java.util.Date());
+ handleSQLException(warn);
+ }
+
+ SQLWarning next = warn.getNextWarning();
+ if (next != warn) {
+ showWarnings(next);
+ }
+ }
+
+
+ String getPrompt() {
+ if (getDatabaseConnection() == null || getDatabaseConnection().getUrl() == null) {
+ return "beeline> ";
+ } else {
+ return getPrompt(getDatabaseConnections().getIndex()
+ + ": " + getDatabaseConnection().getUrl()) + "> ";
+ }
+ }
+
+
+ static String getPrompt(String url) {
+ if (url == null || url.length() == 0) {
+ url = "beeline";
+ }
+ if (url.indexOf(";") > -1) {
+ url = url.substring(0, url.indexOf(";"));
+ }
+ if (url.indexOf("?") > -1) {
+ url = url.substring(0, url.indexOf("?"));
+ }
+ if (url.length() > 45) {
+ url = url.substring(0, 45);
+ }
+ return url;
+ }
+
+
+ /**
+ * Try to obtain the current size of the specified {@link ResultSet} by jumping to the last row
+ * and getting the row number.
+ *
+ * @param rs
+ * the {@link ResultSet} to get the size for
+ * @return the size, or -1 if it could not be obtained
+ */
+ int getSize(ResultSet rs) {
+ try {
+ if (rs.getType() == rs.TYPE_FORWARD_ONLY) {
+ return -1;
+ }
+ rs.last();
+ int total = rs.getRow();
+ rs.beforeFirst();
+ return total;
+ } catch (SQLException sqle) {
+ return -1;
+ }
+ // JDBC 1 driver error
+ catch (AbstractMethodError ame) {
+ return -1;
+ }
+ }
+
+
+ ResultSet getColumns(String table) throws SQLException {
+ if (!(assertConnection())) {
+ return null;
+ }
+ return getDatabaseConnection().getDatabaseMetaData().getColumns(
+ getDatabaseConnection().getDatabaseMetaData().getConnection().getCatalog(), null, table, "%");
+ }
+
+
+ ResultSet getTables() throws SQLException {
+ if (!(assertConnection())) {
+ return null;
+ }
+ return getDatabaseConnection().getDatabaseMetaData().getTables(
+ getDatabaseConnection().getDatabaseMetaData().getConnection().getCatalog(), null, "%",
+ new String[] {"TABLE"});
+ }
+
+
+ String[] getColumnNames(DatabaseMetaData meta) throws SQLException {
+ Set names = new HashSet();
+ info(loc("building-tables"));
+ try {
+ ResultSet columns = getColumns("%");
+ try {
+ int total = getSize(columns);
+ int index = 0;
+
+ while (columns.next()) {
+ // add the following strings:
+ // 1. column name
+ // 2. table name
+ // 3. tablename.columnname
+
+ progress(index++, total);
+ String name = columns.getString("TABLE_NAME");
+ names.add(name);
+ names.add(columns.getString("COLUMN_NAME"));
+ names.add(columns.getString("TABLE_NAME") + "."
+ + columns.getString("COLUMN_NAME"));
+ }
+ progress(index, index);
+ } finally {
+ columns.close();
+ }
+ info(loc("done"));
+ return names.toArray(new String[0]);
+ } catch (Throwable t) {
+ handleException(t);
+ return new String[0];
+ }
+ }
+
+
+ // //////////////////
+ // String utilities
+ // //////////////////
+
+
+ /**
+ * Split the line into an array by tokenizing on space characters
+ *
+ * @param line
+ * the line to break up
+ * @return an array of individual words
+ */
+ String[] split(String line) {
+ return split(line, " ");
+ }
+
+
+ String dequote(String str) {
+ if (str == null) {
+ return null;
+ }
+ while ((str.startsWith("'") && str.endsWith("'"))
+ || (str.startsWith("\"") && str.endsWith("\""))) {
+ str = str.substring(1, str.length() - 1);
+ }
+ return str;
+ }
+
+
+ String[] split(String line, String delim) {
+ StringTokenizer tok = new StringTokenizer(line, delim);
+ String[] ret = new String[tok.countTokens()];
+ int index = 0;
+ while (tok.hasMoreTokens()) {
+ String t = tok.nextToken();
+ t = dequote(t);
+ ret[index++] = t;
+ }
+ return ret;
+ }
+
+
+ static Map map(Object[] obs) {
+ Map m = new HashMap();
+ for (int i = 0; i < obs.length - 1; i += 2) {
+ m.put(obs[i], obs[i + 1]);
+ }
+ return Collections.unmodifiableMap(m);
+ }
+
+
+ static boolean getMoreResults(Statement stmnt) {
+ try {
+ return stmnt.getMoreResults();
+ } catch (Throwable t) {
+ return false;
+ }
+ }
+
+
+ static String xmlattrencode(String str) {
+ str = replace(str, "\"", """);
+ str = replace(str, "<", "<");
+ return str;
+ }
+
+
+ static String replace(String source, String from, String to) {
+ if (source == null) {
+ return null;
+ }
+
+ if (from.equals(to)) {
+ return source;
+ }
+
+ StringBuilder replaced = new StringBuilder();
+
+ int index = -1;
+ while ((index = source.indexOf(from)) != -1) {
+ replaced.append(source.substring(0, index));
+ replaced.append(to);
+ source = source.substring(index + from.length());
+ }
+ replaced.append(source);
+
+ return replaced.toString();
+ }
+
+
+ /**
+ * Split the line based on spaces, asserting that the
+ * number of words is correct.
+ *
+ * @param line
+ * the line to split
+ * @param assertLen
+ * the number of words to assure
+ * @param usage
+ * the message to output if there are an incorrect
+ * number of words.
+ * @return the split lines, or null if the assertion failed.
+ */
+ String[] split(String line, int assertLen, String usage) {
+ String[] ret = split(line);
+
+ if (ret.length != assertLen) {
+ error(usage);
+ return null;
+ }
+
+ return ret;
+ }
+
+
+ /**
+ * Wrap the specified string by breaking on space characters.
+ *
+ * @param toWrap
+ * the string to wrap
+ * @param len
+ * the maximum length of any line
+ * @param start
+ * the number of spaces to pad at the
+ * beginning of a line
+ * @return the wrapped string
+ */
+ String wrap(String toWrap, int len, int start) {
+ StringBuilder buff = new StringBuilder();
+ StringBuilder line = new StringBuilder();
+
+ char[] head = new char[start];
+ Arrays.fill(head, ' ');
+
+ for (StringTokenizer tok = new StringTokenizer(toWrap, " "); tok.hasMoreTokens();) {
+ String next = tok.nextToken();
+ if (line.length() + next.length() > len) {
+ buff.append(line).append(separator).append(head);
+ line.setLength(0);
+ }
+
+ line.append(line.length() == 0 ? "" : " ").append(next);
+ }
+
+ buff.append(line);
+ return buff.toString();
+ }
+
+
+ /**
+ * Output a progress indicator to the console.
+ *
+ * @param cur
+ * the current progress
+ * @param max
+ * the maximum progress, or -1 if unknown
+ */
+ void progress(int cur, int max) {
+ StringBuilder out = new StringBuilder();
+
+ if (lastProgress != null) {
+ char[] back = new char[lastProgress.length()];
+ Arrays.fill(back, '\b');
+ out.append(back);
+ }
+
+ String progress = cur + "/"
+ + (max == -1 ? "?" : "" + max) + " "
+ + (max == -1 ? "(??%)"
+ : ("(" + (cur * 100 / (max == 0 ? 1 : max)) + "%)"));
+
+ if (cur >= max && max != -1) {
+ progress += " " + loc("done") + separator;
+ lastProgress = null;
+ } else {
+ lastProgress = progress;
+ }
+
+ out.append(progress);
+
+ outputStream.print(out.toString());
+ outputStream.flush();
+ }
+
+ // /////////////////////////////
+ // Exception handling routines
+ // /////////////////////////////
+
+ void handleException(Throwable e) {
+ while (e instanceof InvocationTargetException) {
+ e = ((InvocationTargetException) e).getTargetException();
+ }
+
+ if (e instanceof SQLException) {
+ handleSQLException((SQLException) e);
+ } else if (!(getOpts().getVerbose())) {
+ if (e.getMessage() == null) {
+ error(e.getClass().getName());
+ } else {
+ error(e.getMessage());
+ }
+ } else {
+ e.printStackTrace(getErrorStream());
+ }
+ }
+
+
+ void handleSQLException(SQLException e) {
+ if (e instanceof SQLWarning && !(getOpts().getShowWarnings())) {
+ return;
+ }
+
+ error(loc(e instanceof SQLWarning ? "Warning" : "Error",
+ new Object[] {
+ e.getMessage() == null ? "" : e.getMessage().trim(),
+ e.getSQLState() == null ? "" : e.getSQLState().trim(),
+ new Integer(e.getErrorCode())}));
+
+ if (getOpts().getVerbose()) {
+ e.printStackTrace(getErrorStream());
+ }
+
+ if (!getOpts().getShowNestedErrs()) {
+ return;
+ }
+
+ for (SQLException nested = e.getNextException(); nested != null && nested != e; nested = nested
+ .getNextException()) {
+ handleSQLException(nested);
+ }
+ }
+
+
+ boolean scanForDriver(String url) {
+ try {
+ // already registered
+ if (findRegisteredDriver(url) != null) {
+ return true;
+ }
+
+ // first try known drivers...
+ scanDrivers(true);
+
+ if (findRegisteredDriver(url) != null) {
+ return true;
+ }
+
+ // now really scan...
+ scanDrivers(false);
+
+ if (findRegisteredDriver(url) != null) {
+ return true;
+ }
+
+ return false;
+ } catch (Exception e) {
+ debug(e.toString());
+ return false;
+ }
+ }
+
+
+ private Driver findRegisteredDriver(String url) {
+ for (Enumeration drivers = DriverManager.getDrivers(); drivers != null
+ && drivers.hasMoreElements();) {
+ Driver driver = (Driver) drivers.nextElement();
+ try {
+ if (driver.acceptsURL(url)) {
+ return driver;
+ }
+ } catch (Exception e) {
+ }
+ }
+ return null;
+ }
+
+
+ Driver[] scanDrivers(String line) throws IOException {
+ return scanDrivers(false);
+ }
+
+
+ Driver[] scanDrivers(boolean knownOnly) throws IOException {
+ long start = System.currentTimeMillis();
+
+ Set classNames = new HashSet();
+
+ if (!knownOnly) {
+ classNames.addAll(Arrays.asList(
+ ClassNameCompletor.getClassNames()));
+ }
+
+ classNames.addAll(KNOWN_DRIVERS);
+
+ Set driverClasses = new HashSet();
+
+ for (Iterator i = classNames.iterator(); i.hasNext();) {
+ String className = i.next().toString();
+
+ if (className.toLowerCase().indexOf("driver") == -1) {
+ continue;
+ }
+
+ try {
+ Class c = Class.forName(className, false,
+ Thread.currentThread().getContextClassLoader());
+ if (!Driver.class.isAssignableFrom(c)) {
+ continue;
+ }
+
+ if (Modifier.isAbstract(c.getModifiers())) {
+ continue;
+ }
+
+ // now instantiate and initialize it
+ driverClasses.add(c.newInstance());
+ } catch (Throwable t) {
+ }
+ }
+ info("scan complete in "
+ + (System.currentTimeMillis() - start) + "ms");
+ return (Driver[]) driverClasses.toArray(new Driver[0]);
+ }
+
+
+ private Driver[] scanDriversOLD(String line) {
+ long start = System.currentTimeMillis();
+
+ Set paths = new HashSet();
+ Set driverClasses = new HashSet();
+
+ for (StringTokenizer tok = new StringTokenizer(
+ System.getProperty("java.ext.dirs"),
+ System.getProperty("path.separator")); tok.hasMoreTokens();) {
+ File[] files = new File(tok.nextToken()).listFiles();
+ for (int i = 0; files != null && i < files.length; i++) {
+ paths.add(files[i].getAbsolutePath());
+ }
+ }
+
+ for (StringTokenizer tok = new StringTokenizer(
+ System.getProperty("java.class.path"),
+ System.getProperty("path.separator")); tok.hasMoreTokens();) {
+ paths.add(new File(tok.nextToken()).getAbsolutePath());
+ }
+
+ for (Iterator i = paths.iterator(); i.hasNext();) {
+ File f = new File(i.next());
+ output(getColorBuffer().pad(loc("scanning", f.getAbsolutePath()), 60),
+ false);
+
+ try {
+ ZipFile zf = new ZipFile(f);
+ int total = zf.size();
+ int index = 0;
+
+ for (Enumeration zfEnum = zf.entries(); zfEnum.hasMoreElements();) {
+ ZipEntry entry = (ZipEntry) zfEnum.nextElement();
+ String name = entry.getName();
+ progress(index++, total);
+
+ if (name.endsWith(".class")) {
+ name = name.replace('/', '.');
+ name = name.substring(0, name.length() - 6);
+
+ try {
+ // check for the string "driver" in the class
+ // to see if we should load it. Not perfect, but
+ // it is far too slow otherwise.
+ if (name.toLowerCase().indexOf("driver") != -1) {
+ Class c = Class.forName(name, false,
+ getClass().getClassLoader());
+ if (Driver.class.isAssignableFrom(c)
+ && !(Modifier.isAbstract(
+ c.getModifiers()))) {
+ try {
+ // load and initialize
+ Class.forName(name);
+ } catch (Exception e) {
+ }
+ driverClasses.add(c.newInstance());
+ }
+ }
+ } catch (Throwable t) {
+ }
+ }
+ }
+ progress(total, total);
+ } catch (Exception e) {
+ }
+ }
+
+ info("scan complete in "
+ + (System.currentTimeMillis() - start) + "ms");
+ return (Driver[]) driverClasses.toArray(new Driver[0]);
+ }
+
+
+ // /////////////////////////////////////
+ // ResultSet output formatting classes
+ // /////////////////////////////////////
+
+
+
+ int print(ResultSet rs) throws SQLException {
+ String format = getOpts().getOutputFormat();
+ OutputFormat f = (OutputFormat) formats.get(format);
+
+ if (f == null) {
+ error(loc("unknown-format", new Object[] {
+ format, formats.keySet()}));
+ f = new TableOutputFormat(this);
+ }
+
+ Rows rows;
+
+ if (getOpts().getIncremental()) {
+ rows = new IncrementalRows(this, rs);
+ } else {
+ rows = new BufferedRows(this, rs);
+ }
+ return f.print(rows);
+ }
+
+
+ Statement createStatement() throws SQLException {
+ Statement stmnt = getDatabaseConnection().getConnection().createStatement();
+ if (getOpts().timeout > -1) {
+ stmnt.setQueryTimeout(getOpts().timeout);
+ }
+ if (signalHandler != null) {
+ signalHandler.setStatement(stmnt);
+ }
+ return stmnt;
+ }
+
+
+ void runBatch(List statements) {
+ try {
+ Statement stmnt = createStatement();
+ try {
+ for (Iterator i = statements.iterator(); i.hasNext();) {
+ stmnt.addBatch(i.next().toString());
+ }
+ int[] counts = stmnt.executeBatch();
+
+ output(getColorBuffer().pad(getColorBuffer().bold("COUNT"), 8)
+ .append(getColorBuffer().bold("STATEMENT")));
+
+ for (int i = 0; counts != null && i < counts.length; i++) {
+ output(getColorBuffer().pad(counts[i] + "", 8)
+ .append(statements.get(i).toString()));
+ }
+ } finally {
+ try {
+ stmnt.close();
+ } catch (Exception e) {
+ }
+ }
+ } catch (Exception e) {
+ handleException(e);
+ }
+ }
+
+ public int runCommands(String[] cmds) {
+ return runCommands(Arrays.asList(cmds));
+ }
+
+ public int runCommands(List cmds) {
+ int successCount = 0;
+ try {
+ // TODO: Make script output prefixing configurable. Had to disable this since
+ // it results in lots of test diffs.
+ for (String cmd : cmds) {
+ info(getColorBuffer().pad(SCRIPT_OUTPUT_PREFIX, SCRIPT_OUTPUT_PAD_SIZE).append(cmd));
+ // if we do not force script execution, abort
+ // when a failure occurs.
+ if (dispatch(cmd) || getOpts().getForce()) {
+ ++successCount;
+ } else {
+ error(loc("abort-on-error", cmd));
+ return successCount;
+ }
+ }
+ } catch (Exception e) {
+ handleException(e);
+ }
+ return successCount;
+ }
+
+ // ////////////////////////
+ // Command methods follow
+ // ////////////////////////
+
+ void setCompletions() throws SQLException, IOException {
+ if (getDatabaseConnection() != null) {
+ getDatabaseConnection().setCompletions(getOpts().getFastConnect());
+ }
+ }
+
+ BeeLineOpts getOpts() {
+ return opts;
+ }
+
+ DatabaseConnections getDatabaseConnections() {
+ return connections;
+ }
+
+ Completor getCommandCompletor() {
+ return beeLineCommandCompletor;
+ }
+
+ public boolean isExit() {
+ return exit;
+ }
+
+ public void setExit(boolean exit) {
+ this.exit = exit;
+ }
+
+ Collection getDrivers() {
+ return drivers;
+ }
+
+ void setDrivers(Collection drivers) {
+ this.drivers = drivers;
+ }
+
+ public static String getSeparator() {
+ return separator;
+ }
+
+ Commands getCommands() {
+ return commands;
+ }
+
+ OutputFile getScriptOutputFile() {
+ return scriptOutputFile;
+ }
+
+ void setScriptOutputFile(OutputFile script) {
+ this.scriptOutputFile = script;
+ }
+
+ OutputFile getRecordOutputFile() {
+ return recordOutputFile;
+ }
+
+ void setRecordOutputFile(OutputFile record) {
+ this.recordOutputFile = record;
+ }
+
+ public void setOutputStream(PrintStream outputStream) {
+ this.outputStream = new PrintStream(outputStream, true);
+ }
+
+ PrintStream getOutputStream() {
+ return outputStream;
+ }
+
+ public void setErrorStream(PrintStream errorStream) {
+ this.errorStream = new PrintStream(errorStream, true);
+ }
+
+ PrintStream getErrorStream() {
+ return errorStream;
+ }
+
+ ConsoleReader getConsoleReader() {
+ return consoleReader;
+ }
+
+ void setConsoleReader(ConsoleReader reader) {
+ this.consoleReader = reader;
+ }
+
+ List getBatch() {
+ return batch;
+ }
+
+ void setBatch(List batch) {
+ this.batch = batch;
+ }
+
+ protected Reflector getReflector() {
+ return reflector;
+ }
+}
diff --git beeline/src/java/org/apache/hive/beeline/BeeLine.properties beeline/src/java/org/apache/hive/beeline/BeeLine.properties
new file mode 100644
index 0000000..fa02f92
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/BeeLine.properties
@@ -0,0 +1,162 @@
+app-introduction: {0} version {1} by {2}
+
+jline-version: The version of the required {0} library is too old. Version \
+ "{1}" was found, but "{2}" is required.
+
+enter-for-more: [ Hit "enter" for more ("q" to exit) ]
+no-manual: Could not find manual resource.
+executing-command: Executing command: {0}
+unknown-command: Unknown command: {0}
+autocommit-needs-off: Operation requires that autocommit be turned off.
+no-current-connection: No current connection
+connection-is-closed: Connection is closed
+reconnecting: Reconnecting to "{0}"...
+connecting: Connecting to "{0}"...
+no-driver: No known driver to handle "{0}"
+setting-prop: Setting property: {0}
+saving-options: Saving preferences to: {0}
+loaded-options: Loaded preferences from: {0}
+
+jdbc-level: JDBC level
+compliant: Compliant
+jdbc-version: Version
+driver-class: Driver Class
+
+help-quit: Exits the program
+help-dropall: Drop all tables in the current database
+help-connect: Open a new connection to the database.
+help-manual: Display the BeeLine manual
+help-typeinfo: Display the type map for the current connection
+help-describe: Describe a table
+help-reconnect: Reconnect to the database
+help-metadata: Obtain metadata information
+help-dbinfo: Give metadata information about the database
+help-rehash: Fetch table and column names for command completion
+help-verbose: Set verbose mode on
+help-run: Run a script from the specified file
+help-list: List the current connections
+help-all: Execute the specified SQL against all the current connections
+help-go: Select the current connection
+help-script: Start saving a script to a file
+help-brief: Set verbose mode off
+help-close: Close the current connection to the database
+help-closeall: Close all current open connections
+help-isolation: Set the transaction isolation for this connection
+help-nativesql: Show the native SQL for the specified statement
+help-call: Execute a callable statement
+help-autocommit: Set autocommit mode on or off
+help-commit: Commit the current transaction (if autocommit is off)
+help-rollback: Roll back the current transaction (if autocommit is off)
+help-batch: Start or execute a batch of statements
+help-help: Print a summary of command usage
+help-set: Set a beeline variable
+help-save: Save the current variabes and aliases
+help-native: Show the database''s native SQL for a command
+help-alias: Create a new command alias
+help-unalias: Unset a command alias
+help-scan: Scan for installed JDBC drivers
+help-sql: Execute a SQL command
+help-history: Display the command history
+help-record: Record all output to the specified file
+help-indexes: List all the indexes for the specified table
+help-primarykeys: List all the primary keys for the specified table
+help-exportedkeys: List all the exported keys for the specified table
+help-importedkeys: List all the imported keys for the specified table
+help-procedures: List all the procedures
+help-tables: List all the tables in the database
+help-columns: List all the columns for the specified table
+help-properties: Connect to the database specified in the properties file(s)
+help-outputformat: Set the output format for displaying results (table,vertical,csv,tsv,xmlattrs,xmlelements)
+
+jline-missing: The JLine jar was not found. Please ensure it is installed.
+
+batch-start: Batching SQL statements. Run "batch" again to execute the batch.
+running-batch: Running batched SQL statements...
+
+arg-usage: Usage: {0} <{1}>
+
+scanning: Scanning {0}...
+no-such-method: No such method "{0}"
+possible-methods: Possible methods:
+
+closing: Closing: {0}
+already-closed: Connection is already closed.
+error-setting: Error setting configuration: {0}: {1}
+no-method: No method matching "{0}" was found in {1}.
+
+
+connected: Connected to: {0} (version {1})
+driver: Driver: {0} (version {1})
+autocommit-status: Autocommit status: {0}
+isolation-status: Transaction isolation: {0}
+unknown-format: Unknown output format "{0}". Possible values: {1}
+
+closed: closed
+open: open
+
+executing-con: Executing SQL against: {0}
+comments: Comments, bug reports, and patches go to {0}
+building-tables: Building list of tables and columns for tab-completion \
+ (set fastconnect to true to skip)...
+done: Done
+state: state
+code: code
+
+invalid-connections: Invalid connection: {0}
+
+script-closed: Script closed. Enter "run {0}" to replay it.
+script-already-running: Script ({0}) is already running. Enter "script" with no arguments to stop it.
+script-started: Saving command script to "{0}". Enter "script" with no arguments to stop it.
+
+
+record-closed: Recording stopped.
+record-already-running: Output already being saved to ({0}). Enter "record" with no arguments to stop it.
+record-started: Saving all output to "{0}". Enter "record" with no arguments to stop it.
+
+autoloading-known-drivers: No known driver to handle "{0}". Searching for known drivers...
+
+Warning: Warning: {0} (state={1},code={2,number,#})
+Error: Error: {0} (state={1},code={2,number,#})
+
+commit-complete: Commit complete
+rollback-complete: Rollback complete
+
+abort-on-error: Aborting command set because "force" is false and \
+ command failed: "{0}"
+
+multiple-matches: Ambiguous command: {0}
+
+really-drop-all: Really drop every table in the database? (y/n)\
+abort-drop-all: Aborting drop all tables.
+
+drivers-found-count: 0#No driver classes found|1#{0} driver class found|1<{0} driver classes found
+rows-selected: 0#No rows selected|1#{0} row selected|1<{0} rows selected
+rows-affected: 0#No rows affected|1#{0} row affected|1<{0} rows affected|0>Unknown rows affected
+active-connections: 0#No active connections|1#{0} active connection:|1<{0} active connections:
+
+time-ms: ({0,number,#.###} seconds)
+
+cmd-usage: Usage: java org.apache.hive.cli.beeline.BeeLine \n \
+\ -u the JDBC URL to connect to\n \
+\ -n the username to connect as\n \
+\ -p the password to connect as\n \
+\ -d the driver class to use\n \
+\ --color=[true/false] control whether color is used for display\n \
+\ --showHeader=[true/false] show column names in query results\n \
+\ --headerInterval=ROWS; the interval between which heades are displayed\n \
+\ --fastConnect=[true/false] skip building table/column list for tab-completion\n \
+\ --autoCommit=[true/false] enable/disable automatic transaction commit\n \
+\ --verbose=[true/false] show verbose error messages and debug info\n \
+\ --showWarnings=[true/false] display connection warnings\n \
+\ --showNestedErrs=[true/false] display nested errors\n \
+\ --numberFormat=[pattern] format numbers using DecimalFormat pattern\n \
+\ --force=[true/false] continue running script even after errors\n \
+\ --maxWidth=MAXWIDTH the maximum width of the terminal\n \
+\ --maxColumnWidth=MAXCOLWIDTH the maximum width to use when displaying columns\n \
+\ --silent=[true/false] be more silent\n \
+\ --autosave=[true/false] automatically save preferences\n \
+\ --outputformat=[table/vertical/csv/tsv] format mode for result display\n \
+\ --isolation=LEVEL set the transaction isolation level\n \
+\ --help display this message
+
+
diff --git beeline/src/java/org/apache/hive/beeline/BeeLineCommandCompletor.java beeline/src/java/org/apache/hive/beeline/BeeLineCommandCompletor.java
new file mode 100644
index 0000000..a0bfb17
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/BeeLineCommandCompletor.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+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/BeeLineCompletor.java beeline/src/java/org/apache/hive/beeline/BeeLineCompletor.java
new file mode 100644
index 0000000..746466e
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/BeeLineCompletor.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+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
new file mode 100644
index 0000000..0a06876
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
@@ -0,0 +1,437 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Properties;
+import java.util.TreeSet;
+
+import jline.Completor;
+import jline.SimpleCompletor;
+import jline.Terminal;
+
+
+class BeeLineOpts implements Completor {
+ public static final int DEFAULT_MAX_WIDTH = 80;
+ public static final int DEFAULT_MAX_HEIGHT = 80;
+ public static final int DEFAULT_HEADER_INTERVAL = 100;
+ public static final String DEFAULT_ISOLATION_LEVEL =
+ "TRANSACTION_REPEATABLE_READ";
+ public static final String PROPERTY_PREFIX = "beeline.";
+ public static final String PROPERTY_NAME_EXIT =
+ PROPERTY_PREFIX + "system.exit";
+
+ private final BeeLine beeLine;
+ private boolean autosave = false;
+ private boolean silent = false;
+ private boolean color = false;
+ private boolean showHeader = true;
+ private int headerInterval = 100;
+ private boolean fastConnect = true;
+ private boolean autoCommit = false;
+ private boolean verbose = false;
+ private boolean force = false;
+ private boolean incremental = false;
+ private boolean showWarnings = false;
+ private boolean showNestedErrs = false;
+ private boolean showElapsedTime = true;
+ private String numberFormat = "default";
+ private final Terminal terminal = Terminal.setupTerminal();
+ private int maxWidth = DEFAULT_MAX_WIDTH;
+ private int maxHeight = DEFAULT_MAX_HEIGHT;
+ private int maxColumnWidth = 15;
+ int timeout = -1;
+ private String isolation = DEFAULT_ISOLATION_LEVEL;
+ private String outputFormat = "table";
+ private boolean trimScripts = true;
+
+ private final File rcFile = new File(saveDir(), "beeline.properties");
+ private String historyFile = new File(saveDir(), "history").getAbsolutePath();
+
+
+ public BeeLineOpts(BeeLine beeLine, Properties props) {
+ this.beeLine = beeLine;
+ if (terminal.getTerminalWidth() > 0) {
+ maxWidth = terminal.getTerminalWidth();
+ }
+ if (terminal.getTerminalHeight() > 0) {
+ maxHeight = terminal.getTerminalHeight();
+ }
+ loadProperties(props);
+ }
+
+
+ public Completor[] optionCompletors() {
+ return new Completor[] {this};
+ }
+
+ public String[] possibleSettingValues() {
+ List vals = new LinkedList();
+ vals.addAll(Arrays.asList(new String[] {"yes", "no"}));
+ return vals.toArray(new String[vals.size()]);
+ }
+
+
+ /**
+ * The save directory if HOME/.beeline/ on UNIX, and
+ * HOME/beeline/ on Windows.
+ */
+ public File saveDir() {
+ String dir = System.getProperty("beeline.rcfile");
+ if (dir != null && dir.length() > 0) {
+ return new File(dir);
+ }
+
+ File f = new File(System.getProperty("user.home"),
+ (System.getProperty("os.name").toLowerCase()
+ .indexOf("windows") != -1 ? "" : ".") + "beeline")
+ .getAbsoluteFile();
+ try {
+ f.mkdirs();
+ } catch (Exception e) {
+ }
+ return f;
+ }
+
+
+ @Override
+ public int complete(String buf, int pos, List cand) {
+ try {
+ return new SimpleCompletor(propertyNames()).complete(buf, pos, cand);
+ } catch (Throwable t) {
+ return -1;
+ }
+ }
+
+
+ public void save() throws IOException {
+ OutputStream out = new FileOutputStream(rcFile);
+ save(out);
+ out.close();
+ }
+
+ public void save(OutputStream out) throws IOException {
+ try {
+ Properties props = toProperties();
+ // don't save maxwidth: it is automatically set based on
+ // the terminal configuration
+ props.remove(PROPERTY_PREFIX + "maxwidth");
+ props.store(out, beeLine.getApplicationTitle());
+ } catch (Exception e) {
+ beeLine.handleException(e);
+ }
+ }
+
+ String[] propertyNames()
+ throws IllegalAccessException, InvocationTargetException {
+ TreeSet names = new TreeSet();
+
+ // get all the values from getXXX methods
+ Method[] m = getClass().getDeclaredMethods();
+ for (int i = 0; m != null && i < m.length; i++) {
+ if (!(m[i].getName().startsWith("get"))) {
+ continue;
+ }
+ if (m[i].getParameterTypes().length != 0) {
+ continue;
+ }
+ String propName = m[i].getName().substring(3).toLowerCase();
+ names.add(propName);
+ }
+ return names.toArray(new String[names.size()]);
+ }
+
+
+ public Properties toProperties()
+ throws IllegalAccessException, InvocationTargetException,
+ ClassNotFoundException {
+ Properties props = new Properties();
+
+ String[] names = propertyNames();
+ for (int i = 0; names != null && i < names.length; i++) {
+ props.setProperty(PROPERTY_PREFIX + names[i],
+ beeLine.getReflector().invoke(this, "get" + names[i], new Object[0])
+ .toString());
+ }
+ beeLine.debug("properties: " + props.toString());
+ return props;
+ }
+
+
+ public void load() throws IOException {
+ InputStream in = new FileInputStream(rcFile);
+ load(in);
+ in.close();
+ }
+
+
+ public void load(InputStream fin) throws IOException {
+ Properties p = new Properties();
+ p.load(fin);
+ loadProperties(p);
+ }
+
+
+ public void loadProperties(Properties props) {
+ for (Object element : props.keySet()) {
+ String key = element.toString();
+ if (key.equals(PROPERTY_NAME_EXIT)) {
+ // fix for sf.net bug 879422
+ continue;
+ }
+ if (key.startsWith(PROPERTY_PREFIX)) {
+ set(key.substring(PROPERTY_PREFIX.length()),
+ props.getProperty(key));
+ }
+ }
+ }
+
+ public void set(String key, String value) {
+ set(key, value, false);
+ }
+
+ public boolean set(String key, String value, boolean quiet) {
+ try {
+ beeLine.getReflector().invoke(this, "set" + key, new Object[] {value});
+ return true;
+ } catch (Exception e) {
+ if (!quiet) {
+ beeLine.error(beeLine.loc("error-setting", new Object[] {key, e}));
+ }
+ return false;
+ }
+ }
+
+ public void setFastConnect(boolean fastConnect) {
+ this.fastConnect = fastConnect;
+ }
+
+
+ public boolean getFastConnect() {
+ return fastConnect;
+ }
+
+ public void setAutoCommit(boolean autoCommit) {
+ this.autoCommit = autoCommit;
+ }
+
+ public boolean getAutoCommit() {
+ return autoCommit;
+ }
+
+ public void setVerbose(boolean verbose) {
+ this.verbose = verbose;
+ }
+
+ public boolean getVerbose() {
+ return verbose;
+ }
+
+ public void setShowWarnings(boolean showWarnings) {
+ this.showWarnings = showWarnings;
+ }
+
+ public boolean getShowWarnings() {
+ return showWarnings;
+ }
+
+ public void setShowNestedErrs(boolean showNestedErrs) {
+ this.showNestedErrs = showNestedErrs;
+ }
+
+ public boolean getShowNestedErrs() {
+ return showNestedErrs;
+ }
+
+ public void setShowElapsedTime(boolean showElapsedTime) {
+ this.showElapsedTime = showElapsedTime;
+ }
+
+ public boolean getShowElapsedTime() {
+ return showElapsedTime;
+ }
+
+ public void setNumberFormat(String numberFormat) {
+ this.numberFormat = numberFormat;
+ }
+
+ public String getNumberFormat() {
+ return numberFormat;
+ }
+
+ public void setMaxWidth(int maxWidth) {
+ this.maxWidth = maxWidth;
+ }
+
+ public int getMaxWidth() {
+ return maxWidth;
+ }
+
+ public void setMaxColumnWidth(int maxColumnWidth) {
+ this.maxColumnWidth = maxColumnWidth;
+ }
+
+ public int getMaxColumnWidth() {
+ return maxColumnWidth;
+ }
+
+ public void setTimeout(int timeout) {
+ this.timeout = timeout;
+ }
+
+ public int getTimeout() {
+ return timeout;
+ }
+
+ public void setIsolation(String isolation) {
+ this.isolation = isolation;
+ }
+
+ public String getIsolation() {
+ return isolation;
+ }
+
+ public void setHistoryFile(String historyFile) {
+ this.historyFile = historyFile;
+ }
+
+ public String getHistoryFile() {
+ return historyFile;
+ }
+
+ public void setColor(boolean color) {
+ this.color = color;
+ }
+
+ public boolean getColor() {
+ return color;
+ }
+
+ public void setShowHeader(boolean showHeader) {
+ this.showHeader = showHeader;
+ }
+
+ public boolean getShowHeader() {
+ return showHeader;
+ }
+
+ public void setHeaderInterval(int headerInterval) {
+ this.headerInterval = headerInterval;
+ }
+
+ public int getHeaderInterval() {
+ return headerInterval;
+ }
+
+ public void setForce(boolean force) {
+ this.force = force;
+ }
+
+ public boolean getForce() {
+ return force;
+ }
+
+ public void setIncremental(boolean incremental) {
+ this.incremental = incremental;
+ }
+
+ public boolean getIncremental() {
+ return incremental;
+ }
+
+ public void setSilent(boolean silent) {
+ this.silent = silent;
+ }
+
+ public boolean isSilent() {
+ return silent;
+ }
+
+ public void setAutosave(boolean autosave) {
+ this.autosave = autosave;
+ }
+
+ public boolean getAutosave() {
+ return autosave;
+ }
+
+ public void setOutputFormat(String outputFormat) {
+ this.outputFormat = outputFormat;
+ }
+
+ public String getOutputFormat() {
+ return outputFormat;
+ }
+
+ public void setTrimScripts(boolean trimScripts) {
+ this.trimScripts = trimScripts;
+ }
+
+ public boolean getTrimScripts() {
+ return trimScripts;
+ }
+
+ public void setMaxHeight(int maxHeight) {
+ this.maxHeight = maxHeight;
+ }
+
+ public int getMaxHeight() {
+ return maxHeight;
+ }
+
+ public File getPropertiesFile() {
+ return rcFile;
+ }
+}
diff --git beeline/src/java/org/apache/hive/beeline/BeeLineSignalHandler.java beeline/src/java/org/apache/hive/beeline/BeeLineSignalHandler.java
new file mode 100644
index 0000000..cab521f
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/BeeLineSignalHandler.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+
+package org.apache.hive.beeline;
+
+import java.sql.Statement;
+
+
+/**
+ * BeeLineSignalHandler.
+ *
+ */
+public interface BeeLineSignalHandler {
+ public void setStatement(Statement stmt);
+}
diff --git beeline/src/java/org/apache/hive/beeline/BufferedRows.java beeline/src/java/org/apache/hive/beeline/BufferedRows.java
new file mode 100644
index 0000000..4384a52
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/BufferedRows.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Iterator;
+import java.util.LinkedList;
+
+/**
+ * Rows implementation which buffers all rows in a linked list.
+ */
+class BufferedRows extends Rows {
+ private final LinkedList list;
+ private final Iterator iterator;
+
+ BufferedRows(BeeLine beeLine, ResultSet rs) throws SQLException {
+ super(beeLine, rs);
+ list = new LinkedList();
+ int count = rsMeta.getColumnCount();
+ list.add(new Row(count));
+ while (rs.next()) {
+ list.add(new Row(count, rs));
+ }
+ iterator = list.iterator();
+ }
+
+ public boolean hasNext() {
+ return iterator.hasNext();
+ }
+
+ public Object next() {
+ return iterator.next();
+ }
+
+ @Override
+ void normalizeWidths() {
+ int[] max = null;
+ for (Row row : list) {
+ if (max == null) {
+ max = new int[row.values.length];
+ }
+ for (int j = 0; j < max.length; j++) {
+ max[j] = Math.max(max[j], row.sizes[j] + 1);
+ }
+ }
+ for (Row row : list) {
+ row.sizes = max;
+ }
+ }
+
+}
diff --git beeline/src/java/org/apache/hive/beeline/ColorBuffer.java beeline/src/java/org/apache/hive/beeline/ColorBuffer.java
new file mode 100644
index 0000000..3b2f8d4
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/ColorBuffer.java
@@ -0,0 +1,261 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * A buffer that can output segments using ANSI color.
+ *
+ */
+final class ColorBuffer implements Comparable {
+ private static final ColorBuffer.ColorAttr BOLD = new ColorAttr("\033[1m");
+ private static final ColorBuffer.ColorAttr NORMAL = new ColorAttr("\033[m");
+ private static final ColorBuffer.ColorAttr REVERS = new ColorAttr("\033[7m");
+ private static final ColorBuffer.ColorAttr LINED = new ColorAttr("\033[4m");
+ private static final ColorBuffer.ColorAttr GREY = new ColorAttr("\033[1;30m");
+ private static final ColorBuffer.ColorAttr RED = new ColorAttr("\033[1;31m");
+ private static final ColorBuffer.ColorAttr GREEN = new ColorAttr("\033[1;32m");
+ private static final ColorBuffer.ColorAttr BLUE = new ColorAttr("\033[1;34m");
+ private static final ColorBuffer.ColorAttr CYAN = new ColorAttr("\033[1;36m");
+ private static final ColorBuffer.ColorAttr YELLOW = new ColorAttr("\033[1;33m");
+ private static final ColorBuffer.ColorAttr MAGENTA = new ColorAttr("\033[1;35m");
+ private static final ColorBuffer.ColorAttr INVISIBLE = new ColorAttr("\033[8m");
+
+ private final List parts = new LinkedList();
+ private int visibleLength = 0;
+
+ private final boolean useColor;
+
+
+ public ColorBuffer(boolean useColor) {
+ this.useColor = useColor;
+ append("");
+ }
+
+ public ColorBuffer(String str, boolean useColor) {
+ this.useColor = useColor;
+ append(str);
+ }
+
+ /**
+ * Pad the specified String with spaces to the indicated length
+ *
+ * @param str
+ * the String to pad
+ * @param len
+ * the length we want the return String to be
+ * @return the passed in String with spaces appended until the
+ * length matches the specified length.
+ */
+ ColorBuffer pad(ColorBuffer str, int len) {
+ while (str.getVisibleLength() < len) {
+ str.append(" ");
+ }
+ return append(str);
+ }
+
+ ColorBuffer center(String str, int len) {
+ StringBuilder buf = new StringBuilder(str);
+ while (buf.length() < len) {
+ buf.append(" ");
+ if (buf.length() < len) {
+ buf.insert(0, " ");
+ }
+ }
+ return append(buf.toString());
+ }
+
+ ColorBuffer pad(String str, int len) {
+ if (str == null) {
+ str = "";
+ }
+ return pad(new ColorBuffer(str, false), len);
+ }
+
+ public String getColor() {
+ return getBuffer(useColor);
+ }
+
+ public String getMono() {
+ return getBuffer(false);
+ }
+
+ String getBuffer(boolean color) {
+ StringBuilder buf = new StringBuilder();
+ for (Object part : parts) {
+ if (!color && part instanceof ColorBuffer.ColorAttr) {
+ continue;
+ }
+ buf.append(part.toString());
+ }
+ return buf.toString();
+ }
+
+
+ /**
+ * Truncate the ColorBuffer to the specified length and return
+ * the new ColorBuffer. Any open color tags will be closed.
+ * Do nothing if the specified length is <= 0.
+ */
+ public ColorBuffer truncate(int len) {
+ if (len <= 0) {
+ return this;
+ }
+ ColorBuffer cbuff = new ColorBuffer(useColor);
+ ColorBuffer.ColorAttr lastAttr = null;
+ for (Iterator i = parts.iterator(); cbuff.getVisibleLength() < len && i.hasNext();) {
+ Object next = i.next();
+ if (next instanceof ColorBuffer.ColorAttr) {
+ lastAttr = (ColorBuffer.ColorAttr) next;
+ cbuff.append((ColorBuffer.ColorAttr) next);
+ continue;
+ }
+ String val = next.toString();
+ if (cbuff.getVisibleLength() + val.length() > len) {
+ int partLen = len - cbuff.getVisibleLength();
+ val = val.substring(0, partLen);
+ }
+ cbuff.append(val);
+ }
+
+ // close off the buffer with a normal tag
+ if (lastAttr != null && lastAttr != NORMAL) {
+ cbuff.append(NORMAL);
+ }
+
+ return cbuff;
+ }
+
+
+ @Override
+ public String toString() {
+ return getColor();
+ }
+
+ public ColorBuffer append(String str) {
+ parts.add(str);
+ visibleLength += str.length();
+ return this;
+ }
+
+ public ColorBuffer append(ColorBuffer buf) {
+ parts.addAll(buf.parts);
+ visibleLength += buf.getVisibleLength();
+ return this;
+ }
+
+ private ColorBuffer append(ColorBuffer.ColorAttr attr) {
+ parts.add(attr);
+ return this;
+ }
+
+ public int getVisibleLength() {
+ return visibleLength;
+ }
+
+ private ColorBuffer append(ColorBuffer.ColorAttr attr, String val) {
+ parts.add(attr);
+ parts.add(val);
+ parts.add(NORMAL);
+ visibleLength += val.length();
+ return this;
+ }
+
+ public ColorBuffer bold(String str) {
+ return append(BOLD, str);
+ }
+
+ public ColorBuffer lined(String str) {
+ return append(LINED, str);
+ }
+
+ public ColorBuffer grey(String str) {
+ return append(GREY, str);
+ }
+
+ public ColorBuffer red(String str) {
+ return append(RED, str);
+ }
+
+ public ColorBuffer blue(String str) {
+ return append(BLUE, str);
+ }
+
+ public ColorBuffer green(String str) {
+ return append(GREEN, str);
+ }
+
+ public ColorBuffer cyan(String str) {
+ return append(CYAN, str);
+ }
+
+ public ColorBuffer yellow(String str) {
+ return append(YELLOW, str);
+ }
+
+ public ColorBuffer magenta(String str) {
+ return append(MAGENTA, str);
+ }
+
+ private static class ColorAttr {
+ private final String attr;
+
+ public ColorAttr(String attr) {
+ this.attr = attr;
+ }
+
+ @Override
+ public String toString() {
+ return attr;
+ }
+ }
+
+ public int compareTo(Object other) {
+ return getMono().compareTo(((ColorBuffer) other).getMono());
+ }
+}
\ No newline at end of file
diff --git beeline/src/java/org/apache/hive/beeline/CommandHandler.java beeline/src/java/org/apache/hive/beeline/CommandHandler.java
new file mode 100644
index 0000000..4d09da9
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/CommandHandler.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import jline.Completor;
+
+/**
+ * A generic command to be executed. Execution of the command
+ * should be dispatched to the {@link #execute(java.lang.String)} method after determining that
+ * the command is appropriate with
+ * the {@link #matches(java.lang.String)} method.
+ *
+ */
+interface CommandHandler {
+ /**
+ * @return the name of the command
+ */
+ public String getName();
+
+
+ /**
+ * @return all the possible names of this command.
+ */
+ public String[] getNames();
+
+
+ /**
+ * @return the short help description for this command.
+ */
+ public String getHelpText();
+
+
+ /**
+ * Check to see if the specified string can be dispatched to this
+ * command.
+ *
+ * @param line
+ * the command line to check.
+ * @return the command string that matches, or null if it no match
+ */
+ public String matches(String line);
+
+
+ /**
+ * Execute the specified command.
+ *
+ * @param line
+ * the full command line to execute.
+ */
+ public boolean execute(String line);
+
+
+ /**
+ * Returns the completors that can handle parameters.
+ */
+ public Completor[] getParameterCompletors();
+}
\ No newline at end of file
diff --git beeline/src/java/org/apache/hive/beeline/Commands.java beeline/src/java/org/apache/hive/beeline/Commands.java
new file mode 100644
index 0000000..8e2a52f
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/Commands.java
@@ -0,0 +1,1298 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import java.io.BufferedReader;
+import java.io.FileInputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.lang.reflect.Method;
+import java.sql.CallableStatement;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.Driver;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.TreeSet;
+
+
+public class Commands {
+ private final BeeLine beeLine;
+
+ /**
+ * @param beeLine
+ */
+ Commands(BeeLine beeLine) {
+ this.beeLine = beeLine;
+ }
+
+
+ public boolean metadata(String line) {
+ beeLine.debug(line);
+
+ String[] parts = beeLine.split(line);
+ List params = new LinkedList(Arrays.asList(parts));
+ if (parts == null || parts.length == 0) {
+ return dbinfo("");
+ }
+
+ params.remove(0);
+ params.remove(0);
+ beeLine.debug(params.toString());
+ return metadata(parts[1],
+ params.toArray(new String[0]));
+ }
+
+
+ public boolean metadata(String cmd, String[] args) {
+ try {
+ Method[] m = beeLine.getDatabaseConnection().getDatabaseMetaData().getClass().getMethods();
+ Set methodNames = new TreeSet();
+ Set methodNamesUpper = new TreeSet();
+ for (int i = 0; i < m.length; i++) {
+ methodNames.add(m[i].getName());
+ methodNamesUpper.add(m[i].getName().toUpperCase());
+ }
+
+ if (!methodNamesUpper.contains(cmd.toUpperCase())) {
+ beeLine.error(beeLine.loc("no-such-method", cmd));
+ beeLine.error(beeLine.loc("possible-methods"));
+ for (Iterator i = methodNames.iterator(); i.hasNext();) {
+ beeLine.error(" " + i.next());
+ }
+ return false;
+ }
+
+ Object res = beeLine.getReflector().invoke(beeLine.getDatabaseConnection().getDatabaseMetaData(),
+ DatabaseMetaData.class, cmd, Arrays.asList(args));
+
+ if (res instanceof ResultSet) {
+ ResultSet rs = (ResultSet) res;
+ if (rs != null) {
+ try {
+ beeLine.print(rs);
+ } finally {
+ rs.close();
+ }
+ }
+ } else if (res != null) {
+ beeLine.output(res.toString());
+ }
+ } catch (Exception e) {
+ return beeLine.error(e);
+ }
+
+ return true;
+ }
+
+
+ public boolean history(String line) {
+ List hist = beeLine.getConsoleReader().getHistory().getHistoryList();
+ int index = 1;
+ for (Iterator i = hist.iterator(); i.hasNext(); index++) {
+ beeLine.output(beeLine.getColorBuffer().pad(index + ".", 6)
+ .append(i.next().toString()));
+ }
+ return true;
+ }
+
+
+ String arg1(String line, String paramname) {
+ return arg1(line, paramname, null);
+ }
+
+
+ String arg1(String line, String paramname, String def) {
+ String[] ret = beeLine.split(line);
+
+ if (ret == null || ret.length != 2) {
+ if (def != null) {
+ return def;
+ }
+ throw new IllegalArgumentException(beeLine.loc("arg-usage",
+ new Object[] {ret.length == 0 ? "" : ret[0],
+ paramname}));
+ }
+ return ret[1];
+ }
+
+
+ public boolean indexes(String line) throws Exception {
+ return metadata("getIndexInfo", new String[] {
+ beeLine.getConnection().getCatalog(), null,
+ arg1(line, "table name"),
+ false + "",
+ true + ""});
+ }
+
+
+ public boolean primarykeys(String line) throws Exception {
+ return metadata("getPrimaryKeys", new String[] {
+ beeLine.getConnection().getCatalog(), null,
+ arg1(line, "table name"),});
+ }
+
+
+ public boolean exportedkeys(String line) throws Exception {
+ return metadata("getExportedKeys", new String[] {
+ beeLine.getConnection().getCatalog(), null,
+ arg1(line, "table name"),});
+ }
+
+
+ public boolean importedkeys(String line) throws Exception {
+ return metadata("getImportedKeys", new String[] {
+ beeLine.getConnection().getCatalog(), null,
+ arg1(line, "table name"),});
+ }
+
+
+ public boolean procedures(String line) throws Exception {
+ return metadata("getProcedures", new String[] {
+ beeLine.getConnection().getCatalog(), null,
+ arg1(line, "procedure name pattern", "%"),});
+ }
+
+
+ public boolean tables(String line) throws Exception {
+ return metadata("getTables", new String[] {
+ beeLine.getConnection().getCatalog(), null,
+ arg1(line, "table name", "%"), null});
+ }
+
+
+ public boolean typeinfo(String line) throws Exception {
+ return metadata("getTypeInfo", new String[0]);
+ }
+
+
+ public boolean nativesql(String sql) throws Exception {
+ if (sql.startsWith(BeeLine.COMMAND_PREFIX)) {
+ sql = sql.substring(1);
+ }
+ if (sql.startsWith("native")) {
+ sql = sql.substring("native".length() + 1);
+ }
+ String nat = beeLine.getDatabaseConnection().getConnection().nativeSQL(sql);
+ beeLine.output(nat);
+ return true;
+ }
+
+
+ public boolean columns(String line) throws Exception {
+ return metadata("getColumns", new String[] {
+ beeLine.getConnection().getCatalog(), null,
+ arg1(line, "table name"), "%"});
+ }
+
+
+ public boolean dropall(String line) {
+ if (beeLine.getDatabaseConnection() == null || beeLine.getDatabaseConnection().getUrl() == null) {
+ return beeLine.error(beeLine.loc("no-current-connection"));
+ }
+ try {
+ if (!(beeLine.getConsoleReader().readLine(beeLine.loc("really-drop-all")).equals("y"))) {
+ return beeLine.error("abort-drop-all");
+ }
+
+ List cmds = new LinkedList();
+ ResultSet rs = beeLine.getTables();
+ try {
+ while (rs.next()) {
+ cmds.add("DROP TABLE "
+ + rs.getString("TABLE_NAME") + ";");
+ }
+ } finally {
+ try {
+ rs.close();
+ } catch (Exception e) {
+ }
+ }
+ // run as a batch
+ return beeLine.runCommands(cmds) == cmds.size();
+ } catch (Exception e) {
+ return beeLine.error(e);
+ }
+ }
+
+
+ public boolean reconnect(String line) {
+ if (beeLine.getDatabaseConnection() == null || beeLine.getDatabaseConnection().getUrl() == null) {
+ return beeLine.error(beeLine.loc("no-current-connection"));
+ }
+ beeLine.info(beeLine.loc("reconnecting", beeLine.getDatabaseConnection().getUrl()));
+ try {
+ beeLine.getDatabaseConnection().reconnect();
+ } catch (Exception e) {
+ return beeLine.error(e);
+ }
+ return true;
+ }
+
+
+ public boolean scan(String line) throws IOException {
+ TreeSet names = new TreeSet();
+
+ if (beeLine.getDrivers() == null) {
+ beeLine.setDrivers(Arrays.asList(beeLine.scanDrivers(line)));
+ }
+
+ beeLine.info(beeLine.loc("drivers-found-count", beeLine.getDrivers().size()));
+
+ // unique the list
+ for (Iterator i = beeLine.getDrivers().iterator(); i.hasNext();) {
+ names.add(i.next().getClass().getName());
+ }
+
+ beeLine.output(beeLine.getColorBuffer()
+ .bold(beeLine.getColorBuffer().pad(beeLine.loc("compliant"), 10).getMono())
+ .bold(beeLine.getColorBuffer().pad(beeLine.loc("jdbc-version"), 8).getMono())
+ .bold(beeLine.getColorBuffer(beeLine.loc("driver-class")).getMono()));
+
+ for (Iterator i = names.iterator(); i.hasNext();) {
+ String name = i.next().toString();
+ try {
+ Driver driver = (Driver) Class.forName(name).newInstance();
+ ColorBuffer msg = beeLine.getColorBuffer()
+ .pad(driver.jdbcCompliant() ? "yes" : "no", 10)
+ .pad(driver.getMajorVersion() + "."
+ + driver.getMinorVersion(), 8)
+ .append(name);
+ if (driver.jdbcCompliant()) {
+ beeLine.output(msg);
+ } else {
+ beeLine.output(beeLine.getColorBuffer().red(msg.getMono()));
+ }
+ } catch (Throwable t) {
+ beeLine.output(beeLine.getColorBuffer().red(name)); // error with driver
+ }
+ }
+ return true;
+ }
+
+
+ public boolean save(String line) throws IOException {
+ beeLine.info(beeLine.loc("saving-options", beeLine.getOpts().getPropertiesFile()));
+ beeLine.getOpts().save();
+ return true;
+ }
+
+
+ public boolean load(String line) throws IOException {
+ beeLine.getOpts().load();
+ beeLine.info(beeLine.loc("loaded-options", beeLine.getOpts().getPropertiesFile()));
+ return true;
+ }
+
+
+ public boolean config(String line) {
+ try {
+ Properties props = beeLine.getOpts().toProperties();
+ Set keys = new TreeSet(props.keySet());
+ for (Iterator i = keys.iterator(); i.hasNext();) {
+ String key = (String) i.next();
+ beeLine.output(beeLine.getColorBuffer()
+ .green(beeLine.getColorBuffer().pad(key.substring(
+ beeLine.getOpts().PROPERTY_PREFIX.length()), 20)
+ .getMono())
+ .append(props.getProperty(key)));
+ }
+ } catch (Exception e) {
+ return beeLine.error(e);
+ }
+ return true;
+ }
+
+
+ public boolean set(String line) {
+ if (line == null || line.trim().equals("set")
+ || line.length() == 0) {
+ return config(null);
+ }
+
+ String[] parts = beeLine.split(line, 3, "Usage: set ");
+ if (parts == null) {
+ return false;
+ }
+
+ String key = parts[1];
+ String value = parts[2];
+ boolean success = beeLine.getOpts().set(key, value, false);
+ // if we autosave, then save
+ if (success && beeLine.getOpts().getAutosave()) {
+ try {
+ beeLine.getOpts().save();
+ } catch (Exception saveException) {
+ }
+ }
+ return success;
+ }
+
+
+ public boolean commit(String line) throws SQLException {
+ if (!(beeLine.assertConnection())) {
+ return false;
+ }
+ if (!(beeLine.assertAutoCommit())) {
+ return false;
+ }
+ try {
+ long start = System.currentTimeMillis();
+ beeLine.getDatabaseConnection().getConnection().commit();
+ long end = System.currentTimeMillis();
+ beeLine.showWarnings();
+ beeLine.info(beeLine.loc("commit-complete")
+ + " " + beeLine.locElapsedTime(end - start));
+ return true;
+ } catch (Exception e) {
+ return beeLine.error(e);
+ }
+ }
+
+
+ public boolean rollback(String line) throws SQLException {
+ if (!(beeLine.assertConnection())) {
+ return false;
+ }
+ if (!(beeLine.assertAutoCommit())) {
+ return false;
+ }
+ try {
+ long start = System.currentTimeMillis();
+ beeLine.getDatabaseConnection().getConnection().rollback();
+ long end = System.currentTimeMillis();
+ beeLine.showWarnings();
+ beeLine.info(beeLine.loc("rollback-complete")
+ + " " + beeLine.locElapsedTime(end - start));
+ return true;
+ } catch (Exception e) {
+ return beeLine.error(e);
+ }
+ }
+
+
+ public boolean autocommit(String line) throws SQLException {
+ if (!(beeLine.assertConnection())) {
+ return false;
+ }
+ if (line.endsWith("on")) {
+ beeLine.getDatabaseConnection().getConnection().setAutoCommit(true);
+ } else if (line.endsWith("off")) {
+ beeLine.getDatabaseConnection().getConnection().setAutoCommit(false);
+ }
+ beeLine.showWarnings();
+ beeLine.autocommitStatus(beeLine.getDatabaseConnection().getConnection());
+ return true;
+ }
+
+
+ public boolean dbinfo(String line) {
+ if (!(beeLine.assertConnection())) {
+ return false;
+ }
+
+ beeLine.showWarnings();
+ int padlen = 50;
+
+ String[] m = new String[] {
+ "allProceduresAreCallable",
+ "allTablesAreSelectable",
+ "dataDefinitionCausesTransactionCommit",
+ "dataDefinitionIgnoredInTransactions",
+ "doesMaxRowSizeIncludeBlobs",
+ "getCatalogSeparator",
+ "getCatalogTerm",
+ "getDatabaseProductName",
+ "getDatabaseProductVersion",
+ "getDefaultTransactionIsolation",
+ "getDriverMajorVersion",
+ "getDriverMinorVersion",
+ "getDriverName",
+ "getDriverVersion",
+ "getExtraNameCharacters",
+ "getIdentifierQuoteString",
+ "getMaxBinaryLiteralLength",
+ "getMaxCatalogNameLength",
+ "getMaxCharLiteralLength",
+ "getMaxColumnNameLength",
+ "getMaxColumnsInGroupBy",
+ "getMaxColumnsInIndex",
+ "getMaxColumnsInOrderBy",
+ "getMaxColumnsInSelect",
+ "getMaxColumnsInTable",
+ "getMaxConnections",
+ "getMaxCursorNameLength",
+ "getMaxIndexLength",
+ "getMaxProcedureNameLength",
+ "getMaxRowSize",
+ "getMaxSchemaNameLength",
+ "getMaxStatementLength",
+ "getMaxStatements",
+ "getMaxTableNameLength",
+ "getMaxTablesInSelect",
+ "getMaxUserNameLength",
+ "getNumericFunctions",
+ "getProcedureTerm",
+ "getSchemaTerm",
+ "getSearchStringEscape",
+ "getSQLKeywords",
+ "getStringFunctions",
+ "getSystemFunctions",
+ "getTimeDateFunctions",
+ "getURL",
+ "getUserName",
+ "isCatalogAtStart",
+ "isReadOnly",
+ "nullPlusNonNullIsNull",
+ "nullsAreSortedAtEnd",
+ "nullsAreSortedAtStart",
+ "nullsAreSortedHigh",
+ "nullsAreSortedLow",
+ "storesLowerCaseIdentifiers",
+ "storesLowerCaseQuotedIdentifiers",
+ "storesMixedCaseIdentifiers",
+ "storesMixedCaseQuotedIdentifiers",
+ "storesUpperCaseIdentifiers",
+ "storesUpperCaseQuotedIdentifiers",
+ "supportsAlterTableWithAddColumn",
+ "supportsAlterTableWithDropColumn",
+ "supportsANSI92EntryLevelSQL",
+ "supportsANSI92FullSQL",
+ "supportsANSI92IntermediateSQL",
+ "supportsBatchUpdates",
+ "supportsCatalogsInDataManipulation",
+ "supportsCatalogsInIndexDefinitions",
+ "supportsCatalogsInPrivilegeDefinitions",
+ "supportsCatalogsInProcedureCalls",
+ "supportsCatalogsInTableDefinitions",
+ "supportsColumnAliasing",
+ "supportsConvert",
+ "supportsCoreSQLGrammar",
+ "supportsCorrelatedSubqueries",
+ "supportsDataDefinitionAndDataManipulationTransactions",
+ "supportsDataManipulationTransactionsOnly",
+ "supportsDifferentTableCorrelationNames",
+ "supportsExpressionsInOrderBy",
+ "supportsExtendedSQLGrammar",
+ "supportsFullOuterJoins",
+ "supportsGroupBy",
+ "supportsGroupByBeyondSelect",
+ "supportsGroupByUnrelated",
+ "supportsIntegrityEnhancementFacility",
+ "supportsLikeEscapeClause",
+ "supportsLimitedOuterJoins",
+ "supportsMinimumSQLGrammar",
+ "supportsMixedCaseIdentifiers",
+ "supportsMixedCaseQuotedIdentifiers",
+ "supportsMultipleResultSets",
+ "supportsMultipleTransactions",
+ "supportsNonNullableColumns",
+ "supportsOpenCursorsAcrossCommit",
+ "supportsOpenCursorsAcrossRollback",
+ "supportsOpenStatementsAcrossCommit",
+ "supportsOpenStatementsAcrossRollback",
+ "supportsOrderByUnrelated",
+ "supportsOuterJoins",
+ "supportsPositionedDelete",
+ "supportsPositionedUpdate",
+ "supportsSchemasInDataManipulation",
+ "supportsSchemasInIndexDefinitions",
+ "supportsSchemasInPrivilegeDefinitions",
+ "supportsSchemasInProcedureCalls",
+ "supportsSchemasInTableDefinitions",
+ "supportsSelectForUpdate",
+ "supportsStoredProcedures",
+ "supportsSubqueriesInComparisons",
+ "supportsSubqueriesInExists",
+ "supportsSubqueriesInIns",
+ "supportsSubqueriesInQuantifieds",
+ "supportsTableCorrelationNames",
+ "supportsTransactions",
+ "supportsUnion",
+ "supportsUnionAll",
+ "usesLocalFilePerTable",
+ "usesLocalFiles",
+ };
+
+ for (int i = 0; i < m.length; i++) {
+ try {
+ beeLine.output(beeLine.getColorBuffer().pad(m[i], padlen).append(
+ "" + beeLine.getReflector().invoke(beeLine.getDatabaseConnection().getDatabaseMetaData(),
+ m[i], new Object[0])));
+ } catch (Exception e) {
+ beeLine.handleException(e);
+ }
+ }
+ return true;
+ }
+
+
+ public boolean verbose(String line) {
+ beeLine.info("verbose: on");
+ return set("set verbose true");
+ }
+
+
+ public boolean outputformat(String line) {
+ return set("set " + line);
+ }
+
+
+ public boolean brief(String line) {
+ beeLine.info("verbose: off");
+ return set("set verbose false");
+ }
+
+
+ public boolean isolation(String line) throws SQLException {
+ if (!(beeLine.assertConnection())) {
+ return false;
+ }
+
+ int i;
+
+ if (line.endsWith("TRANSACTION_NONE")) {
+ i = Connection.TRANSACTION_NONE;
+ } else if (line.endsWith("TRANSACTION_READ_COMMITTED")) {
+ i = Connection.TRANSACTION_READ_COMMITTED;
+ } else if (line.endsWith("TRANSACTION_READ_UNCOMMITTED")) {
+ i = Connection.TRANSACTION_READ_UNCOMMITTED;
+ } else if (line.endsWith("TRANSACTION_REPEATABLE_READ")) {
+ i = Connection.TRANSACTION_REPEATABLE_READ;
+ } else if (line.endsWith("TRANSACTION_SERIALIZABLE")) {
+ i = Connection.TRANSACTION_SERIALIZABLE;
+ } else {
+ return beeLine.error("Usage: isolation ");
+ }
+
+ beeLine.getDatabaseConnection().getConnection().setTransactionIsolation(i);
+
+ int isol = beeLine.getDatabaseConnection().getConnection().getTransactionIsolation();
+ final String isoldesc;
+ switch (i)
+ {
+ case Connection.TRANSACTION_NONE:
+ isoldesc = "TRANSACTION_NONE";
+ break;
+ case Connection.TRANSACTION_READ_COMMITTED:
+ isoldesc = "TRANSACTION_READ_COMMITTED";
+ break;
+ case Connection.TRANSACTION_READ_UNCOMMITTED:
+ isoldesc = "TRANSACTION_READ_UNCOMMITTED";
+ break;
+ case Connection.TRANSACTION_REPEATABLE_READ:
+ isoldesc = "TRANSACTION_REPEATABLE_READ";
+ break;
+ case Connection.TRANSACTION_SERIALIZABLE:
+ isoldesc = "TRANSACTION_SERIALIZABLE";
+ break;
+ default:
+ isoldesc = "UNKNOWN";
+ }
+
+ beeLine.info(beeLine.loc("isolation-status", isoldesc));
+ return true;
+ }
+
+
+ public boolean batch(String line) {
+ if (!(beeLine.assertConnection())) {
+ return false;
+ }
+ if (beeLine.getBatch() == null) {
+ beeLine.setBatch(new LinkedList());
+ beeLine.info(beeLine.loc("batch-start"));
+ return true;
+ } else {
+ beeLine.info(beeLine.loc("running-batch"));
+ try {
+ beeLine.runBatch(beeLine.getBatch());
+ return true;
+ } catch (Exception e) {
+ return beeLine.error(e);
+ } finally {
+ beeLine.setBatch(null);
+ }
+ }
+ }
+
+ public boolean sql(String line) {
+ return execute(line, false);
+ }
+
+ public boolean call(String line) {
+ return execute(line, true);
+ }
+
+ private boolean execute(String line, boolean call) {
+ if (line == null || line.length() == 0) {
+ return false; // ???
+ }
+
+ // ### FIXME: doing the multi-line handling down here means
+ // higher-level logic never sees the extra lines. So,
+ // for example, if a script is being saved, it won't include
+ // the continuation lines! This is logged as sf.net
+ // bug 879518.
+
+ // use multiple lines for statements not terminated by ";"
+ try {
+ while (!(line.trim().endsWith(";"))) {
+ StringBuilder prompt = new StringBuilder(beeLine.getPrompt());
+ for (int i = 0; i < prompt.length() - 1; i++) {
+ if (prompt.charAt(i) != '>') {
+ prompt.setCharAt(i, i % 2 == 0 ? '.' : ' ');
+ }
+ }
+
+ String extra = beeLine.getConsoleReader().readLine(prompt.toString());
+ if (!beeLine.isComment(extra)) {
+ line += " " + extra;
+ }
+ }
+ } catch (Exception e) {
+ beeLine.handleException(e);
+ }
+
+ if (line.endsWith(";")) {
+ line = line.substring(0, line.length() - 1);
+ }
+
+ if (!(beeLine.assertConnection())) {
+ return false;
+ }
+
+ String sql = line;
+
+ if (sql.startsWith(BeeLine.COMMAND_PREFIX)) {
+ sql = sql.substring(1);
+ }
+
+ String prefix = call ? "call" : "sql";
+
+ if (sql.startsWith(prefix)) {
+ sql = sql.substring(prefix.length());
+ }
+
+ // batch statements?
+ if (beeLine.getBatch() != null) {
+ beeLine.getBatch().add(sql);
+ return true;
+ }
+
+ try {
+ Statement stmnt = null;
+ boolean hasResults;
+
+ try {
+ long start = System.currentTimeMillis();
+
+ if (call) {
+ stmnt = beeLine.getDatabaseConnection().getConnection().prepareCall(sql);
+ hasResults = ((CallableStatement) stmnt).execute();
+ } else {
+ stmnt = beeLine.createStatement();
+ hasResults = stmnt.execute(sql);
+ }
+
+ beeLine.showWarnings();
+
+ if (hasResults) {
+ do {
+ ResultSet rs = stmnt.getResultSet();
+ try {
+ int count = beeLine.print(rs);
+ long end = System.currentTimeMillis();
+
+ beeLine.info(beeLine.loc("rows-selected", count) + " "
+ + beeLine.locElapsedTime(end - start));
+ } finally {
+ rs.close();
+ }
+ } while (BeeLine.getMoreResults(stmnt));
+ } else {
+ int count = stmnt.getUpdateCount();
+ long end = System.currentTimeMillis();
+ beeLine.info(beeLine.loc("rows-affected", count)
+ + " " + beeLine.locElapsedTime(end - start));
+ }
+ } catch (Exception e) {
+ beeLine.error(e);
+ throw e;
+ } finally {
+ if (stmnt != null) {
+ stmnt.close();
+ }
+ }
+ } catch (Exception e) {
+ return beeLine.error(e);
+ }
+ beeLine.showWarnings();
+ return true;
+ }
+
+
+ public boolean quit(String line) {
+ beeLine.setExit(true);
+ close(null);
+ return true;
+ }
+
+
+ /**
+ * Close all connections.
+ */
+ public boolean closeall(String line) {
+ if (close(null)) {
+ while (close(null)) {
+ ;
+ }
+ return true;
+ }
+ return false;
+ }
+
+
+ /**
+ * Close the current connection.
+ */
+ public boolean close(String line) {
+ if (beeLine.getDatabaseConnection() == null) {
+ return false;
+ }
+ try {
+ if (beeLine.getDatabaseConnection().getConnection() != null
+ && !(beeLine.getDatabaseConnection().getConnection().isClosed())) {
+ beeLine.info(beeLine.loc("closing",
+ beeLine.getDatabaseConnection().getConnection().getClass().getName()));
+ beeLine.getDatabaseConnection().getConnection().close();
+ } else {
+ beeLine.info(beeLine.loc("already-closed"));
+ }
+ } catch (Exception e) {
+ return beeLine.error(e);
+ }
+ beeLine.getDatabaseConnections().remove();
+ return true;
+ }
+
+
+ /**
+ * Connect to the database defined in the specified properties file.
+ */
+ public boolean properties(String line) throws Exception {
+ String example = "";
+ example += "Usage: properties " + BeeLine.getSeparator();
+
+ String[] parts = beeLine.split(line);
+ if (parts.length < 2) {
+ return beeLine.error(example);
+ }
+
+ int successes = 0;
+
+ for (int i = 1; i < parts.length; i++) {
+ Properties props = new Properties();
+ props.load(new FileInputStream(parts[i]));
+ if (connect(props)) {
+ successes++;
+ }
+ }
+
+ if (successes != (parts.length - 1)) {
+ return false;
+ } else {
+ return true;
+ }
+ }
+
+
+ public boolean connect(String line) throws Exception {
+ String example = "Usage: connect [driver]"
+ + BeeLine.getSeparator();
+
+ String[] parts = beeLine.split(line);
+ if (parts == null) {
+ return false;
+ }
+
+ if (parts.length < 2) {
+ return beeLine.error(example);
+ }
+
+ String url = parts.length < 2 ? null : parts[1];
+ String user = parts.length < 3 ? null : parts[2];
+ String pass = parts.length < 4 ? null : parts[3];
+ String driver = parts.length < 5 ? null : parts[4];
+
+ Properties props = new Properties();
+ if (url != null) {
+ props.setProperty("url", url);
+ }
+ if (driver != null) {
+ props.setProperty("driver", driver);
+ }
+ if (user != null) {
+ props.setProperty("user", user);
+ }
+ if (pass != null) {
+ props.setProperty("password", pass);
+ }
+ return connect(props);
+ }
+
+
+ private String getProperty(Properties props, String[] keys) {
+ for (int i = 0; i < keys.length; i++) {
+ String val = props.getProperty(keys[i]);
+ if (val != null) {
+ return val;
+ }
+ }
+
+ for (Iterator i = props.keySet().iterator(); i.hasNext();) {
+ String key = (String) i.next();
+ for (int j = 0; j < keys.length; j++) {
+ if (key.endsWith(keys[j])) {
+ return props.getProperty(key);
+ }
+ }
+ }
+
+ return null;
+ }
+
+
+ public boolean connect(Properties props) throws IOException {
+ String url = getProperty(props, new String[] {
+ "url",
+ "javax.jdo.option.ConnectionURL",
+ "ConnectionURL",
+ });
+ String driver = getProperty(props, new String[] {
+ "driver",
+ "javax.jdo.option.ConnectionDriverName",
+ "ConnectionDriverName",
+ });
+ String username = getProperty(props, new String[] {
+ "user",
+ "javax.jdo.option.ConnectionUserName",
+ "ConnectionUserName",
+ });
+ String password = getProperty(props, new String[] {
+ "password",
+ "javax.jdo.option.ConnectionPassword",
+ "ConnectionPassword",
+ });
+
+ if (url == null || url.length() == 0) {
+ return beeLine.error("Property \"url\" is required");
+ }
+ if (driver == null || driver.length() == 0) {
+ if (!beeLine.scanForDriver(url)) {
+ return beeLine.error(beeLine.loc("no-driver", url));
+ }
+ }
+
+ beeLine.info("Connecting to " + url);
+
+ if (username == null) {
+ username = beeLine.getConsoleReader().readLine("Enter username for " + url + ": ");
+ }
+ if (password == null) {
+ password = beeLine.getConsoleReader().readLine("Enter password for " + url + ": ",
+ new Character('*'));
+ }
+
+ try {
+ beeLine.getDatabaseConnections().setConnection(
+ new DatabaseConnection(beeLine, driver, url, username, password));
+ beeLine.getDatabaseConnection().getConnection();
+
+ beeLine.setCompletions();
+ return true;
+ } catch (SQLException sqle) {
+ return beeLine.error(sqle);
+ } catch (IOException ioe) {
+ return beeLine.error(ioe);
+ }
+ }
+
+
+ public boolean rehash(String line) {
+ try {
+ if (!(beeLine.assertConnection())) {
+ return false;
+ }
+ if (beeLine.getDatabaseConnection() != null) {
+ beeLine.getDatabaseConnection().setCompletions(false);
+ }
+ return true;
+ } catch (Exception e) {
+ return beeLine.error(e);
+ }
+ }
+
+
+ /**
+ * List the current connections
+ */
+ public boolean list(String line) {
+ int index = 0;
+ beeLine.info(beeLine.loc("active-connections", beeLine.getDatabaseConnections().size()));
+
+ for (Iterator i = beeLine.getDatabaseConnections().iterator(); i.hasNext(); index++) {
+ DatabaseConnection c = i.next();
+ boolean closed = false;
+ try {
+ closed = c.getConnection().isClosed();
+ } catch (Exception e) {
+ closed = true;
+ }
+
+ beeLine.output(beeLine.getColorBuffer().pad(" #" + index + "", 5)
+ .pad(closed ? beeLine.loc("closed") : beeLine.loc("open"), 9)
+ .append(c.getUrl()));
+ }
+
+ return true;
+ }
+
+
+ public boolean all(String line) {
+ int index = beeLine.getDatabaseConnections().getIndex();
+ boolean success = true;
+
+ for (int i = 0; i < beeLine.getDatabaseConnections().size(); i++) {
+ beeLine.getDatabaseConnections().setIndex(i);
+ beeLine.output(beeLine.loc("executing-con", beeLine.getDatabaseConnection()));
+ // ### FIXME: this is broken for multi-line SQL
+ success = sql(line.substring("all ".length())) && success;
+ }
+
+ // restore index
+ beeLine.getDatabaseConnections().setIndex(index);
+ return success;
+ }
+
+
+ public boolean go(String line) {
+ String[] parts = beeLine.split(line, 2, "Usage: go ");
+ if (parts == null) {
+ return false;
+ }
+ int index = Integer.parseInt(parts[1]);
+ if (!(beeLine.getDatabaseConnections().setIndex(index))) {
+ beeLine.error(beeLine.loc("invalid-connection", "" + index));
+ list(""); // list the current connections
+ return false;
+ }
+ return true;
+ }
+
+
+ /**
+ * Save or stop saving a script to a file
+ */
+ public boolean script(String line) {
+ if (beeLine.getScriptOutputFile() == null) {
+ return startScript(line);
+ } else {
+ return stopScript(line);
+ }
+ }
+
+
+ /**
+ * Stop writing to the script file and close the script.
+ */
+ private boolean stopScript(String line) {
+ try {
+ beeLine.getScriptOutputFile().close();
+ } catch (Exception e)
+ {
+ beeLine.handleException(e);
+ }
+
+ beeLine.output(beeLine.loc("script-closed", beeLine.getScriptOutputFile()));
+ beeLine.setScriptOutputFile(null);
+ return true;
+ }
+
+
+ /**
+ * Start writing to the specified script file.
+ */
+ private boolean startScript(String line) {
+ if (beeLine.getScriptOutputFile() != null) {
+ return beeLine.error(beeLine.loc("script-already-running", beeLine.getScriptOutputFile()));
+ }
+
+ String[] parts = beeLine.split(line, 2, "Usage: script ");
+ if (parts == null) {
+ return false;
+ }
+
+ try {
+ beeLine.setScriptOutputFile(new OutputFile(parts[1]));
+ beeLine.output(beeLine.loc("script-started", beeLine.getScriptOutputFile()));
+ return true;
+ } catch (Exception e) {
+ return beeLine.error(e);
+ }
+ }
+
+
+ /**
+ * Run a script from the specified file.
+ */
+ public boolean run(String line) {
+ String[] parts = beeLine.split(line, 2, "Usage: run ");
+ if (parts == null) {
+ return false;
+ }
+
+ List cmds = new LinkedList();
+
+ try {
+ BufferedReader reader = new BufferedReader(new FileReader(
+ parts[1]));
+ try {
+ // ### NOTE: fix for sf.net bug 879427
+ StringBuilder cmd = null;
+ for (;;) {
+ String scriptLine = reader.readLine();
+
+ if (scriptLine == null) {
+ break;
+ }
+
+ String trimmedLine = scriptLine.trim();
+ if (beeLine.getOpts().getTrimScripts()) {
+ scriptLine = trimmedLine;
+ }
+
+ if (cmd != null) {
+ // we're continuing an existing command
+ cmd.append(" \n");
+ cmd.append(scriptLine);
+ if (trimmedLine.endsWith(";")) {
+ // this command has terminated
+ cmds.add(cmd.toString());
+ cmd = null;
+ }
+ } else {
+ // we're starting a new command
+ if (beeLine.needsContinuation(scriptLine)) {
+ // multi-line
+ cmd = new StringBuilder(scriptLine);
+ } else {
+ // single-line
+ cmds.add(scriptLine);
+ }
+ }
+ }
+
+ if (cmd != null) {
+ // ### REVIEW: oops, somebody left the last command
+ // unterminated; should we fix it for them or complain?
+ // For now be nice and fix it.
+ cmd.append(";");
+ cmds.add(cmd.toString());
+ }
+ } finally {
+ reader.close();
+ }
+
+ // success only if all the commands were successful
+ return beeLine.runCommands(cmds) == cmds.size();
+ } catch (Exception e) {
+ return beeLine.error(e);
+ }
+ }
+
+
+ /**
+ * Save or stop saving all output to a file.
+ */
+ public boolean record(String line) {
+ if (beeLine.getRecordOutputFile() == null) {
+ return startRecording(line);
+ } else {
+ return stopRecording(line);
+ }
+ }
+
+
+ /**
+ * Stop writing output to the record file.
+ */
+ private boolean stopRecording(String line) {
+ try {
+ beeLine.getRecordOutputFile().close();
+ } catch (Exception e) {
+ beeLine.handleException(e);
+ }
+ beeLine.output(beeLine.loc("record-closed", beeLine.getRecordOutputFile()));
+ beeLine.setRecordOutputFile(null);
+ return true;
+ }
+
+
+ /**
+ * Start writing to the specified record file.
+ */
+ private boolean startRecording(String line) {
+ if (beeLine.getRecordOutputFile() != null) {
+ return beeLine.error(beeLine.loc("record-already-running", beeLine.getRecordOutputFile()));
+ }
+
+ String[] parts = beeLine.split(line, 2, "Usage: record ");
+ if (parts == null) {
+ return false;
+ }
+
+ try {
+ beeLine.setRecordOutputFile(new OutputFile(parts[1]));
+ beeLine.output(beeLine.loc("record-started", beeLine.getRecordOutputFile()));
+ return true;
+ } catch (Exception e) {
+ return beeLine.error(e);
+ }
+ }
+
+
+
+
+ public boolean describe(String line) throws SQLException {
+ String[] table = beeLine.split(line, 2, "Usage: describe ");
+ if (table == null) {
+ return false;
+ }
+
+ ResultSet rs;
+
+ if (table[1].equals("tables")) {
+ rs = beeLine.getTables();
+ } else {
+ rs = beeLine.getColumns(table[1]);
+ }
+
+ if (rs == null) {
+ return false;
+ }
+
+ beeLine.print(rs);
+ rs.close();
+ return true;
+ }
+
+
+ public boolean help(String line) {
+ String[] parts = beeLine.split(line);
+ String cmd = parts.length > 1 ? parts[1] : "";
+ int count = 0;
+ TreeSet clist = new TreeSet();
+
+ for (int i = 0; i < beeLine.commandHandlers.length; i++) {
+ if (cmd.length() == 0 ||
+ Arrays.asList(beeLine.commandHandlers[i].getNames()).contains(cmd)) {
+ clist.add(beeLine.getColorBuffer().pad("!" + beeLine.commandHandlers[i].getName(), 20)
+ .append(beeLine.wrap(beeLine.commandHandlers[i].getHelpText(), 60, 20)));
+ }
+ }
+
+ for (Iterator i = clist.iterator(); i.hasNext();) {
+ beeLine.output(i.next());
+ }
+
+ if (cmd.length() == 0) {
+ beeLine.output("");
+ beeLine.output(beeLine.loc("comments", beeLine.getApplicationContactInformation()));
+ }
+
+ return true;
+ }
+
+
+ public boolean manual(String line) throws IOException {
+ InputStream in = BeeLine.class.getResourceAsStream("manual.txt");
+ if (in == null) {
+ return beeLine.error(beeLine.loc("no-manual"));
+ }
+
+ BufferedReader breader = new BufferedReader(
+ new InputStreamReader(in));
+ String man;
+ int index = 0;
+ while ((man = breader.readLine()) != null) {
+ index++;
+ beeLine.output(man);
+
+ // silly little pager
+ if (index % (beeLine.getOpts().getMaxHeight() - 1) == 0) {
+ String ret = beeLine.getConsoleReader().readLine(beeLine.loc("enter-for-more"));
+ if (ret != null && ret.startsWith("q")) {
+ break;
+ }
+ }
+ }
+ breader.close();
+ return true;
+ }
+}
diff --git beeline/src/java/org/apache/hive/beeline/DatabaseConnection.java beeline/src/java/org/apache/hive/beeline/DatabaseConnection.java
new file mode 100644
index 0000000..68d82ca
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/DatabaseConnection.java
@@ -0,0 +1,318 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import jline.ArgumentCompletor;
+import jline.Completor;
+
+class DatabaseConnection {
+ private final BeeLine beeLine;
+ private Connection connection;
+ private DatabaseMetaData meta;
+ private final String driver;
+ private final String url;
+ private final String username;
+ private final String password;
+ private Schema schema = null;
+ private Completor sqlCompletor = null;
+
+
+ public DatabaseConnection(BeeLine beeLine, String driver, String url,
+ String username, String password) throws SQLException {
+ this.beeLine = beeLine;
+ this.driver = driver;
+ this.url = url;
+ this.username = username;
+ this.password = password;
+ }
+
+
+ @Override
+ public String toString() {
+ return getUrl() + "";
+ }
+
+
+ void setCompletions(boolean skipmeta) throws SQLException, IOException {
+ final String extraNameCharacters =
+ getDatabaseMetaData() == null || getDatabaseMetaData().getExtraNameCharacters() == null ? ""
+ : getDatabaseMetaData().getExtraNameCharacters();
+
+ // setup the completor for the database
+ sqlCompletor = new ArgumentCompletor(
+ new SQLCompletor(beeLine, skipmeta),
+ new ArgumentCompletor.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);
+ if (Character.isWhitespace(c)) {
+ return true;
+ }
+ return !(Character.isLetterOrDigit(c))
+ && c != '_'
+ && extraNameCharacters.indexOf(c) == -1;
+ }
+ });
+
+ // not all argument elements need to hold true
+ ((ArgumentCompletor) sqlCompletor).setStrict(false);
+ }
+
+
+ /**
+ * Connection to the specified data source.
+ *
+ * @param driver
+ * the driver class
+ * @param url
+ * the connection URL
+ * @param username
+ * the username
+ * @param password
+ * the password
+ */
+ boolean connect() throws SQLException {
+ try {
+ if (driver != null && driver.length() != 0) {
+ Class.forName(driver);
+ }
+ } catch (ClassNotFoundException cnfe) {
+ return beeLine.error(cnfe);
+ }
+
+ boolean foundDriver = false;
+ try {
+ foundDriver = DriverManager.getDriver(getUrl()) != null;
+ } catch (Exception e) {
+ }
+
+ try {
+ close();
+ } catch (Exception e) {
+ return beeLine.error(e);
+ }
+
+ setConnection(DriverManager.getConnection(getUrl(), username, password));
+ setDatabaseMetaData(getConnection().getMetaData());
+
+ try {
+ beeLine.info(beeLine.loc("connected", new Object[] {
+ getDatabaseMetaData().getDatabaseProductName(),
+ getDatabaseMetaData().getDatabaseProductVersion()}));
+ } catch (Exception e) {
+ beeLine.handleException(e);
+ }
+
+ try {
+ beeLine.info(beeLine.loc("driver", new Object[] {
+ getDatabaseMetaData().getDriverName(),
+ getDatabaseMetaData().getDriverVersion()}));
+ } catch (Exception e) {
+ beeLine.handleException(e);
+ }
+
+ try {
+ getConnection().setAutoCommit(beeLine.getOpts().getAutoCommit());
+ // TODO: Setting autocommit should not generate an exception as long as it is set to false
+ // beeLine.autocommitStatus(getConnection());
+ } catch (Exception e) {
+ beeLine.handleException(e);
+ }
+
+ try {
+ beeLine.getCommands().isolation("isolation: " + beeLine.getOpts().getIsolation());
+ } catch (Exception e) {
+ beeLine.handleException(e);
+ }
+
+ return true;
+ }
+
+
+ public Connection getConnection() throws SQLException {
+ if (connection != null) {
+ return connection;
+ }
+ connect();
+ return connection;
+ }
+
+
+ public void reconnect() throws Exception {
+ close();
+ getConnection();
+ }
+
+
+ public void close() {
+ try {
+ try {
+ if (connection != null && !connection.isClosed()) {
+ beeLine.output(beeLine.loc("closing", connection));
+ connection.close();
+ }
+ } catch (Exception e) {
+ beeLine.handleException(e);
+ }
+ } finally {
+ setConnection(null);
+ setDatabaseMetaData(null);
+ }
+ }
+
+
+ public String[] getTableNames(boolean force) {
+ Schema.Table[] t = getSchema().getTables();
+ Set names = new TreeSet();
+ for (int i = 0; t != null && i < t.length; i++) {
+ names.add(t[i].getName());
+ }
+ return names.toArray(new String[names.size()]);
+ }
+
+ Schema getSchema() {
+ if (schema == null) {
+ schema = new Schema();
+ }
+ return schema;
+ }
+
+ void setConnection(Connection connection) {
+ this.connection = connection;
+ }
+
+ DatabaseMetaData getDatabaseMetaData() {
+ return meta;
+ }
+
+ void setDatabaseMetaData(DatabaseMetaData meta) {
+ this.meta = meta;
+ }
+
+ String getUrl() {
+ return url;
+ }
+
+ Completor getSQLCompletor() {
+ return sqlCompletor;
+ }
+
+ class Schema {
+ private Table[] tables = null;
+
+ Table[] getTables() {
+ if (tables != null) {
+ return tables;
+ }
+
+ List tnames = new LinkedList();
+
+ try {
+ ResultSet rs = getDatabaseMetaData().getTables(getConnection().getCatalog(),
+ null, "%", new String[] {"TABLE"});
+ try {
+ while (rs.next()) {
+ tnames.add(new Table(rs.getString("TABLE_NAME")));
+ }
+ } finally {
+ try {
+ rs.close();
+ } catch (Exception e) {
+ }
+ }
+ } catch (Throwable t) {
+ }
+ return tables = tnames.toArray(new Table[0]);
+ }
+
+ Table getTable(String name) {
+ Table[] t = getTables();
+ for (int i = 0; t != null && i < t.length; i++) {
+ if (name.equalsIgnoreCase(t[i].getName())) {
+ return t[i];
+ }
+ }
+ return null;
+ }
+
+ class Table {
+ final String name;
+ Column[] columns;
+
+ public Table(String name) {
+ this.name = name;
+ }
+
+
+ public String getName() {
+ return name;
+ }
+
+ class Column {
+ final String name;
+ boolean isPrimaryKey;
+
+ public Column(String name) {
+ this.name = name;
+ }
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git beeline/src/java/org/apache/hive/beeline/DatabaseConnections.java beeline/src/java/org/apache/hive/beeline/DatabaseConnections.java
new file mode 100644
index 0000000..6eb499a
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/DatabaseConnections.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+class DatabaseConnections {
+ private final List connections = new ArrayList();
+ private int index = -1;
+
+ public DatabaseConnection current() {
+ if (index != -1) {
+ return connections.get(index);
+ }
+ return null;
+ }
+
+ public int size() {
+ return connections.size();
+ }
+
+ public Iterator iterator() {
+ return connections.iterator();
+ }
+
+ public void remove() {
+ if (index != -1) {
+ connections.remove(index);
+ }
+ while (index >= connections.size()) {
+ index--;
+ }
+ }
+
+ public void setConnection(DatabaseConnection connection) {
+ if (connections.indexOf(connection) == -1) {
+ connections.add(connection);
+ }
+ index = connections.indexOf(connection);
+ }
+
+ public int getIndex() {
+ return index;
+ }
+
+
+ public boolean setIndex(int index) {
+ if (index < 0 || index >= connections.size()) {
+ return false;
+ }
+ this.index = index;
+ return true;
+ }
+}
\ No newline at end of file
diff --git beeline/src/java/org/apache/hive/beeline/DriverInfo.java beeline/src/java/org/apache/hive/beeline/DriverInfo.java
new file mode 100644
index 0000000..5a115a5
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/DriverInfo.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import java.io.IOException;
+import java.util.Properties;
+
+public class DriverInfo {
+ public String sampleURL;
+
+ public DriverInfo(String name) throws IOException {
+ Properties props = new Properties();
+ props.load(DriverInfo.class.getResourceAsStream(name));
+ fromProperties(props);
+ }
+
+ public DriverInfo(Properties props) {
+ fromProperties(props);
+ }
+
+ public void fromProperties(Properties props) {
+ }
+}
\ No newline at end of file
diff --git beeline/src/java/org/apache/hive/beeline/IncrementalRows.java beeline/src/java/org/apache/hive/beeline/IncrementalRows.java
new file mode 100644
index 0000000..2c0fa35
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/IncrementalRows.java
@@ -0,0 +1,129 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.NoSuchElementException;
+
+/**
+ * Rows implementation which returns rows incrementally from result set
+ * without any buffering.
+ */
+public class IncrementalRows extends Rows {
+ private final ResultSet rs;
+ private final Row labelRow;
+ private final Row maxRow;
+ private Row nextRow;
+ private boolean endOfResult;
+ private boolean normalizingWidths;
+
+
+ IncrementalRows(BeeLine beeLine, ResultSet rs) throws SQLException {
+ super(beeLine, rs);
+ this.rs = rs;
+
+ labelRow = new Row(rsMeta.getColumnCount());
+ maxRow = new Row(rsMeta.getColumnCount());
+ int maxWidth = beeLine.getOpts().getMaxColumnWidth();
+
+ // pre-compute normalization so we don't have to deal
+ // with SQLExceptions later
+ for (int i = 0; i < maxRow.sizes.length; ++i) {
+ // normalized display width is based on maximum of display size
+ // and label size
+ maxRow.sizes[i] = Math.max(
+ maxRow.sizes[i],
+ rsMeta.getColumnDisplaySize(i + 1));
+ maxRow.sizes[i] = Math.min(maxWidth, maxRow.sizes[i]);
+ }
+
+ nextRow = labelRow;
+ endOfResult = false;
+ }
+
+
+ public boolean hasNext() {
+ if (endOfResult) {
+ return false;
+ }
+
+ if (nextRow == null) {
+ try {
+ if (rs.next()) {
+ nextRow = new Row(labelRow.sizes.length, rs);
+
+ if (normalizingWidths) {
+ // perform incremental normalization
+ nextRow.sizes = labelRow.sizes;
+ }
+ } else {
+ endOfResult = true;
+ }
+ } catch (SQLException ex) {
+ throw new RuntimeException(ex.toString());
+ }
+ }
+ return (nextRow != null);
+ }
+
+ public Object next() {
+ if (!hasNext()) {
+ throw new NoSuchElementException();
+ }
+ Object ret = nextRow;
+ nextRow = null;
+ return ret;
+ }
+
+ @Override
+ void normalizeWidths() {
+ // normalize label row
+ labelRow.sizes = maxRow.sizes;
+ // and remind ourselves to perform incremental normalization
+ // for each row as it is produced
+ normalizingWidths = true;
+ }
+}
diff --git beeline/src/java/org/apache/hive/beeline/OutputFile.java beeline/src/java/org/apache/hive/beeline/OutputFile.java
new file mode 100644
index 0000000..8171200
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/OutputFile.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+
+public class OutputFile {
+ final File file;
+ final PrintWriter out;
+
+ public OutputFile(String filename) throws IOException {
+ file = new File(filename);
+ out = new PrintWriter(new FileWriter(file));
+ }
+
+ @Override
+ public String toString() {
+ return file.getAbsolutePath();
+ }
+
+ public void addLine(String command) {
+ out.println(command);
+ }
+
+ public void println(String command) {
+ out.println(command);
+ }
+
+ public void print(String command) {
+ out.print(command);
+ }
+
+ public void close() throws IOException {
+ out.close();
+ }
+}
diff --git beeline/src/java/org/apache/hive/beeline/OutputFormat.java beeline/src/java/org/apache/hive/beeline/OutputFormat.java
new file mode 100644
index 0000000..287b1c7
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/OutputFormat.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+interface OutputFormat {
+ int print(Rows rows);
+}
\ No newline at end of file
diff --git beeline/src/java/org/apache/hive/beeline/ReflectiveCommandHandler.java beeline/src/java/org/apache/hive/beeline/ReflectiveCommandHandler.java
new file mode 100644
index 0000000..a00fb53
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/ReflectiveCommandHandler.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import jline.Completor;
+
+import org.apache.hadoop.fs.shell.Command;
+
+/**
+ * A {@link Command} implementation that uses reflection to
+ * determine the method to dispatch the command.
+ *
+ */
+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);
+ this.beeLine = beeLine;
+ }
+
+ public boolean execute(String line) {
+ try {
+ Object ob = beeLine.getCommands().getClass().getMethod(getName(),
+ new Class[] {String.class})
+ .invoke(beeLine.getCommands(), new Object[] {line});
+ return ob != null && ob instanceof Boolean
+ && ((Boolean) ob).booleanValue();
+ } catch (Throwable e) {
+ return beeLine.error(e);
+ }
+ }
+}
\ No newline at end of file
diff --git beeline/src/java/org/apache/hive/beeline/Reflector.java beeline/src/java/org/apache/hive/beeline/Reflector.java
new file mode 100644
index 0000000..2501046
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/Reflector.java
@@ -0,0 +1,157 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+class Reflector {
+ private final BeeLine beeLine;
+
+ public Reflector(BeeLine beeLine) {
+ this.beeLine = beeLine;
+ }
+
+ public Object invoke(Object on, String method, Object[] args)
+ throws InvocationTargetException, IllegalAccessException,
+ ClassNotFoundException {
+ return invoke(on, method, Arrays.asList(args));
+ }
+
+ public Object invoke(Object on, String method, List args)
+ throws InvocationTargetException, IllegalAccessException,
+ ClassNotFoundException {
+ return invoke(on, on == null ? null : on.getClass(), method, args);
+ }
+
+
+ public Object invoke(Object on, Class defClass,
+ String method, List args)
+ throws InvocationTargetException, IllegalAccessException,
+ ClassNotFoundException {
+ Class c = defClass != null ? defClass : on.getClass();
+ List candidateMethods = new LinkedList();
+
+ Method[] m = c.getMethods();
+ for (int i = 0; i < m.length; i++) {
+ if (m[i].getName().equalsIgnoreCase(method)) {
+ candidateMethods.add(m[i]);
+ }
+ }
+
+ if (candidateMethods.size() == 0) {
+ throw new IllegalArgumentException(beeLine.loc("no-method",
+ new Object[] {method, c.getName()}));
+ }
+
+ for (Iterator i = candidateMethods.iterator(); i.hasNext();) {
+ Method meth = i.next();
+ Class[] ptypes = meth.getParameterTypes();
+ if (!(ptypes.length == args.size())) {
+ continue;
+ }
+
+ Object[] converted = convert(args, ptypes);
+ if (converted == null) {
+ continue;
+ }
+
+ if (!Modifier.isPublic(meth.getModifiers())) {
+ continue;
+ }
+ return meth.invoke(on, converted);
+ }
+ return null;
+ }
+
+
+ public static Object[] convert(List objects, Class[] toTypes)
+ throws ClassNotFoundException {
+ Object[] converted = new Object[objects.size()];
+ for (int i = 0; i < converted.length; i++) {
+ converted[i] = convert(objects.get(i), toTypes[i]);
+ }
+ return converted;
+ }
+
+
+ public static Object convert(Object ob, Class toType)
+ throws ClassNotFoundException {
+ if (ob == null || ob.toString().equals("null")) {
+ return null;
+ }
+ if (toType == String.class) {
+ return new String(ob.toString());
+ } else if (toType == Byte.class || toType == byte.class) {
+ return new Byte(ob.toString());
+ } else if (toType == Character.class || toType == char.class) {
+ return new Character(ob.toString().charAt(0));
+ } else if (toType == Short.class || toType == short.class) {
+ return new Short(ob.toString());
+ } else if (toType == Integer.class || toType == int.class) {
+ return new Integer(ob.toString());
+ } else if (toType == Long.class || toType == long.class) {
+ return new Long(ob.toString());
+ } else if (toType == Double.class || toType == double.class) {
+ return new Double(ob.toString());
+ } else if (toType == Float.class || toType == float.class) {
+ return new Float(ob.toString());
+ } else if (toType == Boolean.class || toType == boolean.class) {
+ return new Boolean(ob.toString().equals("true")
+ || ob.toString().equals(true + "")
+ || ob.toString().equals("1")
+ || ob.toString().equals("on")
+ || ob.toString().equals("yes"));
+ } else if (toType == Class.class) {
+ return Class.forName(ob.toString());
+ }
+ return null;
+ }
+}
diff --git beeline/src/java/org/apache/hive/beeline/Rows.java beeline/src/java/org/apache/hive/beeline/Rows.java
new file mode 100644
index 0000000..aacbf05
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/Rows.java
@@ -0,0 +1,186 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.text.DecimalFormat;
+import java.text.NumberFormat;
+import java.util.Iterator;
+
+/**
+ * Abstract base class representing a set of rows to be displayed.
+ */
+abstract class Rows implements Iterator {
+ private final BeeLine beeLine;
+ final ResultSetMetaData rsMeta;
+ final Boolean[] primaryKeys;
+ final NumberFormat numberFormat;
+
+ Rows(BeeLine beeLine, ResultSet rs) throws SQLException {
+ this.beeLine = beeLine;
+ rsMeta = rs.getMetaData();
+ int count = rsMeta.getColumnCount();
+ primaryKeys = new Boolean[count];
+ if (beeLine.getOpts().getNumberFormat().equals("default")) {
+ numberFormat = null;
+ } else {
+ numberFormat = new DecimalFormat(beeLine.getOpts().getNumberFormat());
+ }
+ }
+
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * Update all of the rows to have the same size, set to the
+ * maximum length of each column in the Rows.
+ */
+ abstract void normalizeWidths();
+
+ /**
+ * Return whether the specified column (0-based index) is
+ * a primary key. Since this method depends on whether the
+ * JDBC driver property implements {@link ResultSetMetaData#getTableName} (many do not), it
+ * is not reliable for all databases.
+ */
+ boolean isPrimaryKey(int col) {
+ if (primaryKeys[col] != null) {
+ return primaryKeys[col].booleanValue();
+ }
+
+ try {
+ // this doesn't always work, since some JDBC drivers (e.g.,
+ // Oracle's) return a blank string from getTableName.
+ String table = rsMeta.getTableName(col + 1);
+ String column = rsMeta.getColumnName(col + 1);
+
+ if (table == null || table.length() == 0 ||
+ column == null || column.length() == 0) {
+ return (primaryKeys[col] = new Boolean(false)).booleanValue();
+ }
+
+ ResultSet pks = beeLine.getDatabaseConnection().getDatabaseMetaData().getPrimaryKeys(
+ beeLine.getDatabaseConnection().getDatabaseMetaData().getConnection().getCatalog(), null, table);
+
+ try {
+ while (pks.next()) {
+ if (column.equalsIgnoreCase(
+ pks.getString("COLUMN_NAME"))) {
+ return (primaryKeys[col] = new Boolean(true)).booleanValue();
+ }
+ }
+ } finally {
+ pks.close();
+ }
+
+ return (primaryKeys[col] = new Boolean(false)).booleanValue();
+ } catch (SQLException sqle) {
+ return (primaryKeys[col] = new Boolean(false)).booleanValue();
+ }
+ }
+
+
+ class Row {
+ final String[] values;
+ final boolean isMeta;
+ boolean deleted;
+ boolean inserted;
+ boolean updated;
+ int[] sizes;
+
+ Row(int size) throws SQLException {
+ isMeta = true;
+ values = new String[size];
+ sizes = new int[size];
+ for (int i = 0; i < size; i++) {
+ values[i] = rsMeta.getColumnLabel(i + 1);
+ sizes[i] = values[i] == null ? 1 : values[i].length();
+ }
+
+ deleted = false;
+ updated = false;
+ inserted = false;
+ }
+
+
+ Row(int size, ResultSet rs) throws SQLException {
+ isMeta = false;
+ values = new String[size];
+ sizes = new int[size];
+
+ try {
+ deleted = rs.rowDeleted();
+ } catch (Throwable t) {
+ }
+ try {
+ updated = rs.rowUpdated();
+ } catch (Throwable t) {
+ }
+ try {
+ inserted = rs.rowInserted();
+ } catch (Throwable t) {
+ }
+
+ for (int i = 0; i < size; i++) {
+ if (numberFormat != null) {
+ Object o = rs.getObject(i + 1);
+ if (o == null) {
+ values[i] = null;
+ } else if (o instanceof Number) {
+ values[i] = numberFormat.format(o);
+ } else {
+ values[i] = o.toString();
+ }
+ } else {
+ values[i] = rs.getString(i + 1);
+ }
+ sizes[i] = values[i] == null ? 1 : values[i].length();
+ }
+ }
+ }
+}
diff --git beeline/src/java/org/apache/hive/beeline/SQLCompletor.java beeline/src/java/org/apache/hive/beeline/SQLCompletor.java
new file mode 100644
index 0000000..6b2c975
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/SQLCompletor.java
@@ -0,0 +1,113 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+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/SeparatedValuesOutputFormat.java beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java
new file mode 100644
index 0000000..c06f7e0
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+/**
+ * OutputFormat for values separated by a delimiter.
+ *
+ * TODO : Handle character escaping
+ *
+ */
+class SeparatedValuesOutputFormat implements OutputFormat {
+ /**
+ *
+ */
+ private final BeeLine beeLine;
+ private char separator;
+
+ public SeparatedValuesOutputFormat(BeeLine beeLine, char separator) {
+ this.beeLine = beeLine;
+ setSeparator(separator);
+ }
+
+ public int print(Rows rows) {
+ int count = 0;
+ while (rows.hasNext()) {
+ printRow(rows, (Rows.Row) rows.next());
+ count++;
+ }
+ return count - 1; // sans header row
+ }
+
+ public void printRow(Rows rows, Rows.Row row) {
+ String[] vals = row.values;
+ StringBuilder buf = new StringBuilder();
+ for (int i = 0; i < vals.length; i++) {
+ buf.append(buf.length() == 0 ? "" : "" + getSeparator())
+ .append('\'')
+ .append(vals[i] == null ? "" : vals[i])
+ .append('\'');
+ }
+ beeLine.output(buf.toString());
+ }
+
+ public void setSeparator(char separator) {
+ this.separator = separator;
+ }
+
+ public char getSeparator() {
+ return this.separator;
+ }
+}
diff --git beeline/src/java/org/apache/hive/beeline/SunSignalHandler.java beeline/src/java/org/apache/hive/beeline/SunSignalHandler.java
new file mode 100644
index 0000000..befff94
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/SunSignalHandler.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import sun.misc.Signal;
+import sun.misc.SignalHandler;
+
+public class SunSignalHandler implements BeeLineSignalHandler, SignalHandler {
+ private Statement stmt = null;
+
+ SunSignalHandler () {
+ // Interpret Ctrl+C as a request to cancel the currently
+ // executing query.
+ Signal.handle (new Signal ("INT"), this);
+ }
+
+ public void setStatement(Statement stmt) {
+ this.stmt = stmt;
+ }
+
+ public void handle (Signal signal) {
+ try {
+ if (stmt != null) {
+ stmt.cancel();
+ }
+ } catch (SQLException ex) {
+ // ignore
+ }
+ }
+}
diff --git beeline/src/java/org/apache/hive/beeline/TableNameCompletor.java beeline/src/java/org/apache/hive/beeline/TableNameCompletor.java
new file mode 100644
index 0000000..8f9cc42
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/TableNameCompletor.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+import java.util.List;
+
+import jline.Completor;
+import jline.SimpleCompletor;
+
+class TableNameCompletor implements Completor {
+ private final BeeLine beeLine;
+
+ /**
+ * @param beeLine
+ */
+ TableNameCompletor(BeeLine beeLine) {
+ this.beeLine = beeLine;
+ }
+
+ public int complete(String buf, int pos, List cand) {
+ if (beeLine.getDatabaseConnection() == null) {
+ return -1;
+ }
+ return new SimpleCompletor(beeLine.getDatabaseConnection().getTableNames(true))
+ .complete(buf, pos, cand);
+ }
+}
\ No newline at end of file
diff --git beeline/src/java/org/apache/hive/beeline/TableOutputFormat.java beeline/src/java/org/apache/hive/beeline/TableOutputFormat.java
new file mode 100644
index 0000000..2439bf9
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/TableOutputFormat.java
@@ -0,0 +1,162 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+/**
+ * OutputFormat for a pretty, table-like format.
+ *
+ */
+class TableOutputFormat implements OutputFormat {
+ private final BeeLine beeLine;
+ private final StringBuilder sb = new StringBuilder();
+
+ /**
+ * @param beeLine
+ */
+ TableOutputFormat(BeeLine beeLine) {
+ this.beeLine = beeLine;
+ }
+
+ public int print(Rows rows) {
+ int index = 0;
+ ColorBuffer header = null;
+ ColorBuffer headerCols = null;
+ final int width = beeLine.getOpts().getMaxWidth() - 4;
+
+ // normalize the columns sizes
+ rows.normalizeWidths();
+
+ for (; rows.hasNext();) {
+ Rows.Row row = (Rows.Row) rows.next();
+ ColorBuffer cbuf = getOutputString(rows, row);
+ cbuf = cbuf.truncate(width);
+
+ if (index == 0) {
+ sb.setLength(0);
+ for (int j = 0; j < row.sizes.length; j++) {
+ for (int k = 0; k < row.sizes[j]; k++) {
+ sb.append('-');
+ }
+ sb.append("-+-");
+ }
+
+ headerCols = cbuf;
+ header = beeLine.getColorBuffer()
+ .green(sb.toString())
+ .truncate(headerCols.getVisibleLength());
+ }
+
+ if (index == 0 ||
+ (beeLine.getOpts().getHeaderInterval() > 0
+ && index % beeLine.getOpts().getHeaderInterval() == 0
+ && beeLine.getOpts().getShowHeader())) {
+ printRow(header, true);
+ printRow(headerCols, false);
+ printRow(header, true);
+ }
+
+ if (index != 0) {
+ printRow(cbuf, false);
+ }
+ index++;
+ }
+
+ if (header != null && beeLine.getOpts().getShowHeader()) {
+ printRow(header, true);
+ }
+
+ return index - 1;
+ }
+
+ void printRow(ColorBuffer cbuff, boolean header) {
+ if (header) {
+ beeLine.output(beeLine.getColorBuffer().green("+-").append(cbuff).green("-+"));
+ } else {
+ beeLine.output(beeLine.getColorBuffer().green("| ").append(cbuff).green(" |"));
+ }
+ }
+
+ public ColorBuffer getOutputString(Rows rows, Rows.Row row) {
+ return getOutputString(rows, row, " | ");
+ }
+
+
+ ColorBuffer getOutputString(Rows rows, Rows.Row row, String delim) {
+ ColorBuffer buf = beeLine.getColorBuffer();
+
+ for (int i = 0; i < row.values.length; i++) {
+ if (buf.getVisibleLength() > 0) {
+ buf.green(delim);
+ }
+
+ ColorBuffer v;
+
+ if (row.isMeta) {
+ v = beeLine.getColorBuffer().center(row.values[i], row.sizes[i]);
+ if (rows.isPrimaryKey(i)) {
+ buf.cyan(v.getMono());
+ } else {
+ buf.bold(v.getMono());
+ }
+ } else {
+ v = beeLine.getColorBuffer().pad(row.values[i], row.sizes[i]);
+ if (rows.isPrimaryKey(i)) {
+ buf.cyan(v.getMono());
+ } else {
+ buf.append(v.getMono());
+ }
+ }
+ }
+
+ if (row.deleted) {
+ buf = beeLine.getColorBuffer().red(buf.getMono());
+ } else if (row.updated) {
+ buf = beeLine.getColorBuffer().blue(buf.getMono());
+ } else if (row.inserted) {
+ buf = beeLine.getColorBuffer().green(buf.getMono());
+ }
+ return buf;
+ }
+}
\ No newline at end of file
diff --git beeline/src/java/org/apache/hive/beeline/VerticalOutputFormat.java beeline/src/java/org/apache/hive/beeline/VerticalOutputFormat.java
new file mode 100644
index 0000000..7b19504
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/VerticalOutputFormat.java
@@ -0,0 +1,88 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+/**
+ * OutputFormat for vertical column name: value format.
+ *
+ */
+class VerticalOutputFormat implements OutputFormat {
+ private final BeeLine beeLine;
+
+ /**
+ * @param beeLine
+ */
+ VerticalOutputFormat(BeeLine beeLine) {
+ this.beeLine = beeLine;
+ }
+
+ public int print(Rows rows) {
+ int count = 0;
+ Rows.Row header = (Rows.Row) rows.next();
+ while (rows.hasNext()) {
+ printRow(rows, header, (Rows.Row) rows.next());
+ count++;
+ }
+ return count;
+ }
+
+ public void printRow(Rows rows, Rows.Row header, Rows.Row row) {
+ String[] head = header.values;
+ String[] vals = row.values;
+ int headwidth = 0;
+ for (int i = 0; i < head.length && i < vals.length; i++) {
+ headwidth = Math.max(headwidth, head[i].length());
+ }
+
+ headwidth += 2;
+
+ for (int i = 0; i < head.length && i < vals.length; i++) {
+ beeLine.output(beeLine.getColorBuffer().bold(
+ beeLine.getColorBuffer().pad(head[i], headwidth).getMono())
+ .append(vals[i] == null ? "" : vals[i]));
+ }
+ beeLine.output(""); // spacing
+ }
+}
\ No newline at end of file
diff --git beeline/src/java/org/apache/hive/beeline/XMLAttributeOutputFormat.java beeline/src/java/org/apache/hive/beeline/XMLAttributeOutputFormat.java
new file mode 100644
index 0000000..296a328
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/XMLAttributeOutputFormat.java
@@ -0,0 +1,88 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+class XMLAttributeOutputFormat extends AbstractOutputFormat {
+ private final BeeLine beeLine;
+ private final StringBuilder buf = new StringBuilder();
+
+ /**
+ * @param beeLine
+ */
+ XMLAttributeOutputFormat(BeeLine beeLine) {
+ this.beeLine = beeLine;
+ }
+
+ @Override
+ public void printHeader(Rows.Row header) {
+ beeLine.output("");
+ }
+
+
+ @Override
+ public void printFooter(Rows.Row header) {
+ beeLine.output(" ");
+ }
+
+ @Override
+ public void printRow(Rows rows, Rows.Row header, Rows.Row row) {
+ String[] head = header.values;
+ String[] vals = row.values;
+
+ buf.setLength(0);
+ buf.append(" ");
+ beeLine.output(buf.toString());
+ }
+}
\ No newline at end of file
diff --git beeline/src/java/org/apache/hive/beeline/XMLElementOutputFormat.java beeline/src/java/org/apache/hive/beeline/XMLElementOutputFormat.java
new file mode 100644
index 0000000..e187a6d
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/XMLElementOutputFormat.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright (c) 2002,2003,2004,2005 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 the 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.
+ *
+ * This software is hosted by SourceForge.
+ * SourceForge is a trademark of VA Linux Systems, Inc.
+ */
+
+/*
+ * This source file is based on code taken from SQLLine 1.0.2
+ * The license above originally appeared in src/sqlline/SqlLine.java
+ * http://sqlline.sourceforge.net/
+ */
+package org.apache.hive.beeline;
+
+class XMLElementOutputFormat extends AbstractOutputFormat {
+ private final BeeLine beeLine;
+
+ /**
+ * @param beeLine
+ */
+ XMLElementOutputFormat(BeeLine beeLine) {
+ this.beeLine = beeLine;
+ }
+
+ @Override
+ public void printHeader(Rows.Row header) {
+ beeLine.output("");
+ }
+
+ @Override
+ public void printFooter(Rows.Row header) {
+ beeLine.output(" ");
+ }
+
+ @Override
+ public void printRow(Rows rows, Rows.Row header, Rows.Row row) {
+ String[] head = header.values;
+ String[] vals = row.values;
+
+ beeLine.output(" ");
+ for (int i = 0; i < head.length && i < vals.length; i++) {
+ beeLine.output(" <" + head[i] + ">"
+ + (BeeLine.xmlattrencode(vals[i]))
+ + "" + head[i] + ">");
+ }
+ beeLine.output(" ");
+ }
+}
\ No newline at end of file
diff --git beeline/src/java/org/apache/hive/beeline/sql-keywords.properties beeline/src/java/org/apache/hive/beeline/sql-keywords.properties
new file mode 100644
index 0000000..0f1eb65
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/sql-keywords.properties
@@ -0,0 +1 @@
+ABSOLUTE,ACTION,ADD,ALL,ALLOCATE,ALTER,AND,ANY,ARE,AS,ASC,ASSERTION,AT,AUTHORIZATION,AVG,BEGIN,BETWEEN,BIT,BIT_LENGTH,BOTH,BY,CASCADE,CASCADED,CASE,CAST,CATALOG,CHAR,CHARACTER,CHAR_LENGTH,CHARACTER_LENGTH,CHECK,CLOSE,CLUSTER,COALESCE,COLLATE,COLLATION,COLUMN,COMMIT,CONNECT,CONNECTION,CONSTRAINT,CONSTRAINTS,CONTINUE,CONVERT,CORRESPONDING,COUNT,CREATE,CROSS,CURRENT,CURRENT_DATE,CURRENT_TIME,CURRENT_TIMESTAMP,CURRENT_USER,CURSOR,DATE,DAY,DEALLOCATE,DEC,DECIMAL,DECLARE,DEFAULT,DEFERRABLE,DEFERRED,DELETE,DESC,DESCRIBE,DESCRIPTOR,DIAGNOSTICS,DISCONNECT,DISTINCT,DOMAIN,DOUBLE,DROP,ELSE,END,END-EXEC,ESCAPE,EXCEPT,EXCEPTION,EXEC,EXECUTE,EXISTS,EXTERNAL,EXTRACT,FALSE,FETCH,FIRST,FLOAT,FOR,FOREIGN,FOUND,FROM,FULL,GET,GLOBAL,GO,GOTO,GRANT,GROUP,HAVING,HOUR,IDENTITY,IMMEDIATE,IN,INDICATOR,INITIALLY,INNER,INPUT,INSENSITIVE,INSERT,INT,INTEGER,INTERSECT,INTERVAL,INTO,IS,ISOLATION,JOIN,KEY,LANGUAGE,LAST,LEADING,LEFT,LEVEL,LIKE,LOCAL,LOWER,MATCH,MAX,MIN,MINUTE,MODULE,MONTH,NAMES,NATIONAL,NATURAL,NCHAR,NEXT,NO,NOT,NULL,NULLIF,NUMERIC,OCTET_LENGTH,OF,ON,ONLY,OPEN,OPTION,OR,ORDER,OUTER,OUTPUT,OVERLAPS,OVERWRITE,PAD,PARTIAL,PARTITION,POSITION,PRECISION,PREPARE,PRESERVE,PRIMARY,PRIOR,PRIVILEGES,PROCEDURE,PUBLIC,READ,REAL,REFERENCES,RELATIVE,RESTRICT,REVOKE,RIGHT,ROLLBACK,ROWS,SCHEMA,SCROLL,SECOND,SECTION,SELECT,SESSION,SESSION_USER,SET,SIZE,SMALLINT,SOME,SPACE,SQL,SQLCODE,SQLERROR,SQLSTATE,SUBSTRING,SUM,SYSTEM_USER,TABLE,TEMPORARY,THEN,TIME,TIMESTAMP,TIMEZONE_HOUR,TIMEZONE_MINUTE,TO,TRAILING,TRANSACTION,TRANSLATE,TRANSLATION,TRIM,TRUE,UNION,UNIQUE,UNKNOWN,UPDATE,UPPER,USAGE,USER,USING,VALUE,VALUES,VARCHAR,VARYING,VIEW,WHEN,WHENEVER,WHERE,WITH,WORK,WRITE,YEAR,ZONE,ADA,C,CATALOG_NAME,CHARACTER_SET_CATALOG,CHARACTER_SET_NAME,CHARACTER_SET_SCHEMA,CLASS_ORIGIN,COBOL,COLLATION_CATALOG,COLLATION_NAME,COLLATION_SCHEMA,COLUMN_NAME,COMMAND_FUNCTION,COMMITTED,CONDITION_NUMBER,CONNECTION_NAME,CONSTRAINT_CATALOG,CONSTRAINT_NAME,CONSTRAINT_SCHEMA,CURSOR_NAME,DATA,DATETIME_INTERVAL_CODE,DATETIME_INTERVAL_PRECISION,DYNAMIC_FUNCTION,FORTRAN,LENGTH,MESSAGE_LENGTH,MESSAGE_OCTET_LENGTH,MESSAGE_TEXT,MORE,MUMPS,NAME,NULLABLE,NUMBER,PASCAL,PLI,REPEATABLE,RETURNED_LENGTH,RETURNED_OCTET_LENGTH,RETURNED_SQLSTATE,ROW_COUNT,SCALE,SCHEMA_NAME,SERIALIZABLE,SERVER_NAME,SUBCLASS_ORIGIN,TABLE_NAME,TYPE,UNCOMMITTED,UNNAMED
diff --git beeline/src/java/org/apache/hive/beeline/util/QFileClient.java beeline/src/java/org/apache/hive/beeline/util/QFileClient.java
new file mode 100644
index 0000000..b62a883
--- /dev/null
+++ beeline/src/java/org/apache/hive/beeline/util/QFileClient.java
@@ -0,0 +1,316 @@
+/**
+ * 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.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hive.beeline.BeeLine;
+
+/**
+ * QTestClient.
+ *
+ */
+public class QFileClient {
+ private String username;
+ private String password;
+ private String jdbcUrl;
+ private String jdbcDriver;
+
+ private final File hiveRootDirectory;
+ private File qFileDirectory;
+ private File outputDirectory;
+ private File expectedDirectory;
+ private final File scratchDirectory;
+ private final File warehouseDirectory;
+
+ private File testDataDirectory;
+ private File testScriptDirectory;
+
+ private String qFileName;
+ private String testname;
+
+ private File qFile;
+ private File outputFile;
+ private File expectedFile;
+
+ private PrintStream beelineOutputStream;
+
+ private BeeLine beeLine;
+
+ private RegexFilterSet filterSet;
+
+ private boolean hasErrors = false;
+
+ private static Log LOG = LogFactory
+ .getLog(QFileClient.class.getName());
+
+
+ public QFileClient(HiveConf hiveConf, String hiveRootDirectory, String qFileDirectory, String outputDirectory,
+ String expectedDirectory) {
+ this.hiveRootDirectory = new File(hiveRootDirectory);
+ this.qFileDirectory = new File(qFileDirectory);
+ this.outputDirectory = new File(outputDirectory);
+ this.expectedDirectory = new File(expectedDirectory);
+ this.scratchDirectory = new File(hiveConf.getVar(ConfVars.SCRATCHDIR));
+ this.warehouseDirectory = new File(hiveConf.getVar(ConfVars.METASTOREWAREHOUSE));
+ }
+
+
+ private class RegexFilterSet {
+ private final Map regexFilters = new LinkedHashMap();
+
+ public RegexFilterSet addFilter(String regex, String replacement) {
+ regexFilters.put(Pattern.compile(regex), replacement);
+ return this;
+ }
+
+ public String filter(String input) {
+ for (Pattern pattern : regexFilters.keySet()) {
+ input = pattern.matcher(input).replaceAll(regexFilters.get(pattern));
+ }
+ return input;
+ }
+ }
+
+ void initFilterSet() {
+ // Extract the leading four digits from the unix time value.
+ // Use this as a prefix in order to increase the selectivity
+ // of the unix time stamp replacement regex.
+ String currentTimePrefix = Long.toString(System.currentTimeMillis()).substring(0, 4);
+
+ String userName = System.getProperty("user.name");
+
+ String timePattern = "(Mon|Tue|Wed|Thu|Fri|Sat|Sun) "
+ + "(Jan|Feb|Mar|Apr|May|Jun|Jul|Aug|Sep|Oct|Nov|Dec) "
+ + "\\d{2} \\d{2}:\\d{2}:\\d{2} \\w+ 20\\d{2}";
+ String unixTimePattern = "\\D" + currentTimePrefix + "\\d{6}\\D";
+ String unixTimeMillisPattern = "\\D" + currentTimePrefix + "\\d{9}\\D";
+
+ String operatorPattern = "\"(CONDITION|COPY|DEPENDENCY_COLLECTION|DDL"
+ + "|EXPLAIN|FETCH|FIL|FS|FUNCTION|GBY|HASHTABLEDUMMY|HASTTABLESINK|JOIN"
+ + "|LATERALVIEWFORWARD|LIM|LVJ|MAP|MAPJOIN|MAPRED|MAPREDLOCAL|MOVE|OP|RS"
+ + "|SCR|SEL|STATS|TS|UDTF|UNION)_\\d+\"";
+
+ filterSet = new RegexFilterSet()
+ .addFilter(scratchDirectory.toString() + "[\\w\\-/]+", "!!{hive.exec.scratchdir}!!")
+ .addFilter(warehouseDirectory.toString(), "!!{hive.metastore.warehouse.dir}!!")
+ .addFilter(expectedDirectory.toString(), "!!{expectedDirectory}!!")
+ .addFilter(outputDirectory.toString(), "!!{outputDirectory}!!")
+ .addFilter(qFileDirectory.toString(), "!!{qFileDirectory}!!")
+ .addFilter(hiveRootDirectory.toString(), "!!{hive.root}!!")
+ .addFilter("file:/\\w\\S+", "file:/!!ELIDED!!")
+ .addFilter("pfile:/\\w\\S+", "pfile:/!!ELIDED!!")
+ .addFilter("hdfs:/\\w\\S+", "hdfs:/!!ELIDED!!")
+ .addFilter("last_modified_by=\\w+", "last_modified_by=!!ELIDED!!")
+ .addFilter(timePattern, "!!TIMESTAMP!!")
+ .addFilter("(\\D)" + currentTimePrefix + "\\d{6}(\\D)", "$1!!UNIXTIME!!$2")
+ .addFilter("(\\D)" + currentTimePrefix + "\\d{9}(\\D)", "$1!!UNIXTIMEMILLIS!!$2")
+ .addFilter(userName, "!!{user.name}!!")
+ .addFilter(operatorPattern, "\"$1_!!ELIDED!!\"")
+ ;
+ };
+
+ public QFileClient setUsername(String username) {
+ this.username = username;
+ return this;
+ }
+
+ public QFileClient setPassword(String password) {
+ this.password = password;
+ return this;
+ }
+
+ public QFileClient setJdbcUrl(String jdbcUrl) {
+ this.jdbcUrl = jdbcUrl;
+ return this;
+ }
+
+ public QFileClient setJdbcDriver(String jdbcDriver) {
+ this.jdbcDriver = jdbcDriver;
+ return this;
+ }
+
+ public QFileClient setQFileName(String qFileName) {
+ this.qFileName = qFileName;
+ this.qFile = new File(qFileDirectory, qFileName);
+ this.testname = StringUtils.substringBefore(qFileName, ".");
+ expectedFile = new File(expectedDirectory, qFileName + ".out");
+ outputFile = new File(outputDirectory, qFileName + ".out");
+ return this;
+ }
+
+ public QFileClient setQFileDirectory(String qFileDirectory) {
+ this.qFileDirectory = new File(qFileDirectory);
+ return this;
+ }
+
+ public QFileClient setOutputDirectory(String outputDirectory) {
+ this.outputDirectory = new File(outputDirectory);
+ return this;
+ }
+
+ public QFileClient setExpectedDirectory(String expectedDirectory) {
+ this.expectedDirectory = new File(expectedDirectory);
+ return this;
+ }
+
+ public QFileClient setTestDataDirectory(String testDataDirectory) {
+ this.testDataDirectory = new File(testDataDirectory);
+ return this;
+ }
+
+ public QFileClient setTestScriptDirectory(String testScriptDirectory) {
+ this.testScriptDirectory = new File(testScriptDirectory);
+ return this;
+ }
+
+ public boolean hasErrors() {
+ return hasErrors;
+ }
+
+ private void initBeeLine() throws Exception {
+ beeLine = new BeeLine();
+ beelineOutputStream = new PrintStream(new File(outputDirectory, qFileName + ".beeline"));
+ beeLine.setOutputStream(beelineOutputStream);
+ beeLine.setErrorStream(beelineOutputStream);
+ beeLine.runCommands(new String[] {
+ "!set verbose true",
+ "!set shownestederrs true",
+ "!set showwarnings true",
+ "!set showelapsedtime false",
+ "!set maxwidth -1",
+ "!connect " + jdbcUrl + " " + username + " " + password + " " + jdbcDriver,
+ });
+ }
+
+ private void setUp() {
+ beeLine.runCommands(new String[] {
+ "USE default;",
+ "SHOW TABLES;",
+ "DROP DATABASE IF EXISTS `" + testname + "` CASCADE;",
+ "CREATE DATABASE `" + testname + "`;",
+ "USE `" + testname + "`;",
+ "set test.data.dir=" + testDataDirectory + ";",
+ "set test.script.dir=" + testScriptDirectory + ";",
+ "!run " + testScriptDirectory + "/q_test_init.sql",
+ });
+ }
+
+ private void tearDown() {
+ beeLine.runCommands(new String[] {
+ "!set outputformat table",
+ "USE default;",
+ "DROP DATABASE IF EXISTS `" + testname + "` CASCADE;",
+ });
+ }
+
+ private void runQFileTest() throws Exception {
+ hasErrors = false;
+ beeLine.runCommands(new String[] {
+ "!set outputformat csv",
+ "!record " + outputDirectory + "/" + qFileName + ".raw",
+ });
+
+ if (1 != beeLine.runCommands(new String[] { "!run " + qFileDirectory + "/" + qFileName })) {
+ hasErrors = true;
+ }
+
+ beeLine.runCommands(new String[] { "!record" });
+ }
+
+
+ private void filterResults() throws IOException {
+ initFilterSet();
+ String rawOutput = FileUtils.readFileToString(new File(outputDirectory, qFileName + ".raw"));
+ FileUtils.writeStringToFile(outputFile, filterSet.filter(rawOutput));
+ }
+
+ public void cleanup() {
+ if (beeLine != null) {
+ beeLine.runCommands(new String[] {
+ "!quit"
+ });
+ }
+ if (beelineOutputStream != null) {
+ beelineOutputStream.close();
+ }
+ if (hasErrors) {
+ String oldFileName = outputDirectory + "/" + qFileName + ".raw";
+ String newFileName = oldFileName + ".error";
+ try {
+ FileUtils.moveFile(new File(oldFileName), new File(newFileName));
+ } catch (IOException e) {
+ System.out.println("Failed to move '" + oldFileName + "' to '" + newFileName);
+ }
+ }
+ }
+
+
+ public void run() throws Exception {
+ try {
+ initBeeLine();
+ setUp();
+ runQFileTest();
+ tearDown();
+ filterResults();
+ } finally {
+ cleanup();
+ }
+ }
+
+ /**
+ * Does the test have a file with expected results to compare the log against.
+ * False probably indicates that this is a new test and the caller should
+ * copy the log to the expected results directory.
+ * @return
+ */
+ public boolean hasExpectedResults() {
+ return expectedFile.exists();
+ }
+
+ public boolean compareResults() throws IOException {
+ if (!expectedFile.exists()) {
+ LOG.error("Expected results file does not exist: " + expectedFile);
+ return false;
+ }
+ return FileUtils.contentEquals(expectedFile, outputFile);
+ }
+
+ public void overwriteResults() {
+ try {
+ if (expectedFile.exists()) {
+ FileUtils.forceDelete(expectedFile);
+ }
+ FileUtils.copyFileToDirectory(outputFile, expectedDirectory, true);
+ } catch (IOException e) {
+ LOG.error("Failed to overwrite results!", e);
+ }
+ }
+}
diff --git beeline/src/test/.gitignore beeline/src/test/.gitignore
new file mode 100644
index 0000000..5e7613f
--- /dev/null
+++ beeline/src/test/.gitignore
@@ -0,0 +1,2 @@
+# Dummy file to ensure Git creates this directory
+~
diff --git bin/beeline bin/beeline
new file mode 100644
index 0000000..d247c39
--- /dev/null
+++ bin/beeline
@@ -0,0 +1,21 @@
+#!/usr/bin/env bash
+
+# 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.
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/hive --service beeline "$@"
diff --git ql/src/test/results/beelinepositive/.gitignore ql/src/test/results/beelinepositive/.gitignore
new file mode 100644
index 0000000..9bc15bc
--- /dev/null
+++ ql/src/test/results/beelinepositive/.gitignore
@@ -0,0 +1,2 @@
+# Placeholder
+~
\ No newline at end of file
diff --git ql/src/test/templates/TestBeeLineDriver.vm ql/src/test/templates/TestBeeLineDriver.vm
new file mode 100644
index 0000000..fd5a040
--- /dev/null
+++ ql/src/test/templates/TestBeeLineDriver.vm
@@ -0,0 +1,160 @@
+/**
+ * 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.util;
+
+import static org.junit.Assert.fail;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.*;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QTestUtil;
+import org.apache.hive.service.server.HiveServer2;
+import org.apache.hive.testutils.junit.runners.ConcurrentTestRunner;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+@RunWith(ConcurrentTestRunner.class)
+public class $className {
+ private static final String hiveRootDirectory = "$hiveRootDir";
+ private static final String queryDirectory = "$queryDir";
+ private static final String logDirectory = "$logDir";
+ private static final String resultsDirectory = "$resultsDir";
+ private static boolean overwrite = false;
+ private static String scratchDirectory;
+ private static QTestUtil.QTestSetup miniZKCluster = null;
+
+ private static HiveServer2 hiveServer2;
+
+ @BeforeClass
+ public static void beforeClass() throws Exception {
+ HiveConf hiveConf = new HiveConf();
+ hiveConf.logVars(System.err);
+ System.err.flush();
+
+ scratchDirectory = hiveConf.getVar(SCRATCHDIR);
+
+ String testOutputOverwrite = System.getProperty("test.output.overwrite");
+ if (testOutputOverwrite != null && "true".equalsIgnoreCase(testOutputOverwrite)) {
+ overwrite = true;
+ }
+
+ miniZKCluster = new QTestUtil.QTestSetup();
+ miniZKCluster.preTest(hiveConf);
+
+ System.setProperty("hive.zookeeper.quorum",
+ hiveConf.get("hive.zookeeper.quorum"));
+ System.setProperty("hive.zookeeper.client.port",
+ hiveConf.get("hive.zookeeper.client.port"));
+
+ String disableserver = System.getProperty("test.service.disable.server");
+ if (null != disableserver && disableserver.equalsIgnoreCase("true")) {
+ System.err.println("test.service.disable.server=true "
+ + "Skipping HiveServer2 initialization!");
+ return;
+ }
+
+ hiveServer2 = new HiveServer2();
+ hiveServer2.init(hiveConf);
+ System.err.println("Starting HiveServer2...");
+ hiveServer2.start();
+ Thread.sleep(5000);
+ }
+
+
+ @AfterClass
+ public static void afterClass() {
+ try {
+ if (hiveServer2 != null) {
+ System.err.println("Stopping HiveServer2...");
+ hiveServer2.stop();
+ }
+ } catch (Throwable t) {
+ t.printStackTrace();
+ }
+
+ if (miniZKCluster != null) {
+ try {
+ miniZKCluster.tearDown();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+
+ /*
+ public $className() {
+ }
+ */
+
+ protected static void runTest(String qFileName) throws Exception {
+ QFileClient qClient = new QFileClient(new HiveConf(), hiveRootDirectory,
+ queryDirectory, logDirectory, resultsDirectory)
+ .setQFileName(qFileName)
+ .setUsername("user")
+ .setPassword("password")
+ .setJdbcUrl("jdbc:hive2://localhost:10000")
+ .setJdbcDriver("org.apache.hive.jdbc.HiveDriver")
+ .setTestDataDirectory(hiveRootDirectory + "/data/files")
+ .setTestScriptDirectory(hiveRootDirectory + "/data/scripts");
+
+ long startTime = System.currentTimeMillis();
+ System.out.println(">>> STARTED " + qFileName
+ + " (Thread " + Thread.currentThread().getName() + ")");
+ try {
+ qClient.run();
+ } catch (Exception e) {
+ System.err.println(">>> FAILED " + qFileName + " with exception:");
+ e.printStackTrace();
+ throw e;
+ }
+ long elapsedTime = (System.currentTimeMillis() - startTime)/1000;
+ String time = "(" + elapsedTime + "s)";
+
+ if (qClient.compareResults()) {
+ System.out.println(">>> PASSED " + qFileName + " " + time);
+ } else {
+ if (qClient.hasErrors()) {
+ System.err.println(">>> FAILED " + qFileName + " (ERROR) " + time);
+ fail();
+ }
+ if (overwrite) {
+ System.err.println(">>> PASSED " + qFileName + " (OVERWRITE) " + time);
+ qClient.overwriteResults();
+ } else {
+ System.err.println(">>> FAILED " + qFileName + " (DIFF) " + time);
+ fail();
+ }
+ }
+ }
+
+
+#foreach ($qf in $qfiles)
+ #set ($fname = $qf.getName())
+ #set ($eidx = $fname.indexOf('.'))
+ #set ($tname = $fname.substring(0, $eidx))
+ @Test
+ public void testBeeLineDriver_$tname() throws Exception {
+ runTest("$fname");
+ }
+#end
+
+}
+
+
diff --git testutils/build.xml testutils/build.xml
new file mode 100644
index 0000000..558bc50
--- /dev/null
+++ testutils/build.xml
@@ -0,0 +1,24 @@
+
+
+
+
+
+
+
+
+
diff --git testutils/ivy.xml testutils/ivy.xml
new file mode 100644
index 0000000..10690c1
--- /dev/null
+++ testutils/ivy.xml
@@ -0,0 +1,34 @@
+
+
+
+
+
+
+ The Apache Hive (TM) data warehouse software facilitates querying and managing large datasets residing in distributed storage.
+ https://cwiki.apache.org/confluence/display/Hive/Home
+
+
+
+
+
+
+
+
+
+
diff --git testutils/src/java/org/apache/hive/testutils/junit/runners/ConcurrentTestRunner.java testutils/src/java/org/apache/hive/testutils/junit/runners/ConcurrentTestRunner.java
new file mode 100644
index 0000000..3f6cd6a
--- /dev/null
+++ testutils/src/java/org/apache/hive/testutils/junit/runners/ConcurrentTestRunner.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright (c) 2009-2012, toby weston & tempus-fugit committers
+ *
+ * Licensed 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.testutils.junit.runners;
+
+import static java.util.concurrent.Executors.newFixedThreadPool;
+
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hive.testutils.junit.runners.model.ConcurrentScheduler;
+import org.junit.runners.BlockJUnit4ClassRunner;
+import org.junit.runners.model.InitializationError;
+
+/**
+ * Originally taken from com.google.code.tempusfugit.concurrency.ConcurrentTestRunner
+ */
+public class ConcurrentTestRunner extends BlockJUnit4ClassRunner {
+
+ private int numThreads = 1;
+
+ public ConcurrentTestRunner(Class> type) throws InitializationError {
+ super(type);
+
+ String numThreadsProp = System.getProperty("test.concurrency.num.threads");
+ if (numThreadsProp != null) {
+ numThreads = Integer.valueOf(numThreadsProp);
+ }
+
+ setScheduler(new ConcurrentScheduler(newFixedThreadPool(numThreads, new ConcurrentTestRunnerThreadFactory())));
+
+ System.err.println(">>> ConcurrenTestRunner initialize with " + numThreads + " threads");
+ System.err.flush();
+ }
+
+ private static class ConcurrentTestRunnerThreadFactory implements ThreadFactory {
+ private final AtomicLong count = new AtomicLong();
+
+ public Thread newThread(Runnable runnable) {
+ String threadName = ConcurrentTestRunner.class.getSimpleName() + "-Thread-" + count.getAndIncrement();
+ System.err.println(">>> ConcurrentTestRunner.newThread " + threadName);
+ System.err.flush();
+ return new Thread(runnable, threadName);
+ }
+ }
+}
\ No newline at end of file
diff --git testutils/src/java/org/apache/hive/testutils/junit/runners/model/ConcurrentScheduler.java testutils/src/java/org/apache/hive/testutils/junit/runners/model/ConcurrentScheduler.java
new file mode 100644
index 0000000..fa07356
--- /dev/null
+++ testutils/src/java/org/apache/hive/testutils/junit/runners/model/ConcurrentScheduler.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright (c) 2009-2012, toby weston & tempus-fugit committers
+ *
+ * Licensed 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.testutils.junit.runners.model;
+
+import static com.google.code.tempusfugit.concurrency.ExecutorServiceShutdown.shutdown;
+import static com.google.code.tempusfugit.temporal.Duration.days;
+import static java.lang.Boolean.TRUE;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.concurrent.ExecutorService;
+
+import org.junit.runners.model.RunnerScheduler;
+
+/**
+ * Originally taken from com.google.code.tempusfugit.concurrency.ConcurrentScheduler
+ */
+public class ConcurrentScheduler implements RunnerScheduler {
+
+ private final ExecutorService executor;
+ private final OutputStream outputStream;
+
+ public ConcurrentScheduler(ExecutorService executor) {
+ this(executor, System.err);
+ }
+
+ public ConcurrentScheduler(ExecutorService executor, OutputStream outputStream) {
+ this.executor = executor;
+ this.outputStream = outputStream;
+ }
+
+ public void schedule(Runnable childStatement) {
+ executor.submit(childStatement);
+ }
+
+ public void finished() {
+ if (!successful(shutdown(executor).waitingForCompletion(days(365)))) {
+ writeln(outputStream, "scheduler shutdown timed out before tests completed, "
+ + "you may have executors hanging around...");
+ }
+ }
+
+ private Boolean successful(Boolean completed) {
+ return TRUE.equals(completed);
+ }
+
+ private void writeln(OutputStream stream, String string) {
+ try {
+ stream.write(string.getBytes());
+ stream.write(System.getProperty("line.separator").getBytes());
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+}