From 9eaa291612859137ace2282bb0f5b31056998b07 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 11 Mar 2024 11:26:39 +0800 Subject: [PATCH] [KYUUBI #6109] Fork beeline module from Apache Hive 3.1.3 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit # :mag: Description ## Issue References ๐Ÿ”— This is the first step of #6146, to gain a clear commit history, this PR just simply copied the `hive-beeline` module from Apache Hive 3.1.3, with minimal change to pass the tests and manually test basic functionalities, following PRs are going to remove other Hive deps gradually. ## Describe Your Solution ๐Ÿ”ง - Copy source code and test case from Apache Hive 3.1.3 - Drop `org.apache.hive:hive-beeline:3.1.3` - Backport HIVE-21584 to support JDK 9+ - Drop `HiveCli`, `HiveSchemaTool` and `BeelineInPlaceUpdateStream`, and the corresponding test cases - Temporary ignore(will fix later) `TestClientCommandHookFactory#testConnectHook` because of error `NoClassDefFound org/apache/curator/RetryPolicy` - Tune testing code to pass UT ## Types of changes :bookmark: - [ ] Bugfix (non-breaking change which fixes an issue) - [ ] New feature (non-breaking change which adds functionality) - [ ] Breaking change (fix or feature that would cause existing functionality to change) ## Test Plan ๐Ÿงช Minimal changes to pass the unit tests. --- # Checklist ๐Ÿ“ - [x] This patch was not authored or co-authored using [Generative Tooling](https://www.apache.org/legal/generative-tooling.html) **Be nice. Be informative.** Closes #6109 from pan3793/fork-beeline. Closes #6109 885f9fe2b [Cheng Pan] NOTICE a2efa1c41 [Cheng Pan] fix 5bb1cc9f6 [Cheng Pan] Copy from Apache Hive 3.1.3 Authored-by: Cheng Pan Signed-off-by: Cheng Pan --- .rat-excludes | 2 + NOTICE-binary | 3 - kyuubi-hive-beeline/pom.xml | 37 +- .../hive/beeline/AbstractCommandHandler.java | 103 + .../hive/beeline/AbstractOutputFormat.java | 46 + .../java/org/apache/hive/beeline/BeeLine.java | 2385 +++++++++++++++++ .../hive/beeline/BeeLineCommandCompleter.java | 51 + .../apache/hive/beeline/BeeLineCompleter.java | 52 + .../org/apache/hive/beeline/BeeLineOpts.java | 692 +++++ .../hive/beeline/BeeLineSignalHandler.java | 31 + .../apache/hive/beeline/BooleanCompleter.java | 28 + .../org/apache/hive/beeline/BufferedRows.java | 88 + .../beeline/ClientCommandHookFactory.java | 147 + .../org/apache/hive/beeline/ClientHook.java | 33 + .../org/apache/hive/beeline/ColorBuffer.java | 230 ++ .../apache/hive/beeline/CommandHandler.java | 66 + .../org/apache/hive/beeline/Commands.java | 1911 +++++++++++++ .../hive/beeline/DatabaseConnection.java | 346 +++ .../hive/beeline/DatabaseConnections.java | 75 + ...DeprecatedSeparatedValuesOutputFormat.java | 75 + .../org/apache/hive/beeline/DriverInfo.java | 42 + .../apache/hive/beeline/IncrementalRows.java | 100 + .../IncrementalRowsWithNormalization.java | 84 + .../org/apache/hive/beeline/OutputFile.java | 121 + .../org/apache/hive/beeline/OutputFormat.java | 27 + .../beeline/ReflectiveCommandHandler.java | 71 + .../org/apache/hive/beeline/Reflector.java | 127 + .../java/org/apache/hive/beeline/Rows.java | 186 ++ .../org/apache/hive/beeline/SQLCompleter.java | 107 + .../beeline/SeparatedValuesOutputFormat.java | 133 + .../apache/hive/beeline/SunSignalHandler.java | 61 + .../hive/beeline/TableNameCompletor.java | 44 + .../hive/beeline/TableOutputFormat.java | 141 + .../hive/beeline/VerticalOutputFormat.java | 63 + .../beeline/XMLAttributeOutputFormat.java | 63 + .../hive/beeline/XMLElementOutputFormat.java | 55 + .../hive/beeline/cli/CliOptionsProcessor.java | 130 + .../BeelineConfFileParseException.java | 30 + ...eelineHS2ConnectionFileParseException.java | 30 + .../BeelineSiteParseException.java | 30 + .../hs2connection/BeelineSiteParser.java | 141 + .../HS2ConnectionFileParser.java | 78 + .../hs2connection/HS2ConnectionFileUtils.java | 270 ++ .../HiveSiteHS2ConnectionFileParser.java | 171 ++ .../UserHS2ConnectionFileParser.java | 114 + .../src/main/resources/BeeLine.properties | 226 ++ .../main/resources/beeline-log4j2.properties | 45 + .../main/resources/sql-keywords.properties | 1 + .../apache/hive/beeline/ProxyAuthTest.java | 402 +++ .../beeline/TestBeeLineExceptionHandling.java | 67 + .../hive/beeline/TestBeeLineHistory.java | 94 + .../apache/hive/beeline/TestBeeLineOpts.java | 35 + .../hive/beeline/TestBeelineArgParsing.java | 425 +++ .../apache/hive/beeline/TestBufferedRows.java | 126 + .../beeline/TestClientCommandHookFactory.java | 129 + .../org/apache/hive/beeline/TestCommands.java | 70 + .../hive/beeline/TestIncrementalRows.java | 137 + .../apache/hive/beeline/TestShutdownHook.java | 40 + .../hive/beeline/TestTableOutputFormat.java | 113 + .../TestUserHS2ConnectionFileParser.java | 217 ++ .../src/test/resources/DummyDriver.txt | 59 + .../src/test/resources/hive-site.xml | 53 + .../test-hs2-conn-conf-kerberos-http.xml | 48 + .../test-hs2-conn-conf-kerberos-nossl.xml | 32 + .../test-hs2-conn-conf-kerberos-ssl.xml | 40 + .../test-hs2-connection-conf-list.xml | 36 + .../test-hs2-connection-config-noauth.xml | 28 + .../test-hs2-connection-multi-conf-list.xml | 37 + .../test-hs2-connection-zookeeper-config.xml | 32 + 69 files changed, 11297 insertions(+), 15 deletions(-) create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/AbstractCommandHandler.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/AbstractOutputFormat.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/BeeLine.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/BeeLineCommandCompleter.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/BeeLineCompleter.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/BeeLineOpts.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/BeeLineSignalHandler.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/BooleanCompleter.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/BufferedRows.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/ClientCommandHookFactory.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/ClientHook.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/ColorBuffer.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/CommandHandler.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/Commands.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/DatabaseConnection.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/DatabaseConnections.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/DeprecatedSeparatedValuesOutputFormat.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/DriverInfo.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/IncrementalRows.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/IncrementalRowsWithNormalization.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/OutputFile.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/OutputFormat.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/ReflectiveCommandHandler.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/Reflector.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/Rows.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/SQLCompleter.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/SunSignalHandler.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/TableNameCompletor.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/TableOutputFormat.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/VerticalOutputFormat.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/XMLAttributeOutputFormat.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/XMLElementOutputFormat.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/cli/CliOptionsProcessor.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/hs2connection/BeelineConfFileParseException.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/hs2connection/BeelineHS2ConnectionFileParseException.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/hs2connection/BeelineSiteParseException.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/hs2connection/BeelineSiteParser.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileParser.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileUtils.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/hs2connection/HiveSiteHS2ConnectionFileParser.java create mode 100644 kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/hs2connection/UserHS2ConnectionFileParser.java create mode 100644 kyuubi-hive-beeline/src/main/resources/BeeLine.properties create mode 100644 kyuubi-hive-beeline/src/main/resources/beeline-log4j2.properties create mode 100644 kyuubi-hive-beeline/src/main/resources/sql-keywords.properties create mode 100644 kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/ProxyAuthTest.java create mode 100644 kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/TestBeeLineExceptionHandling.java create mode 100644 kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/TestBeeLineHistory.java create mode 100644 kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/TestBeeLineOpts.java create mode 100644 kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/TestBeelineArgParsing.java create mode 100644 kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/TestBufferedRows.java create mode 100644 kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/TestClientCommandHookFactory.java create mode 100644 kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/TestCommands.java create mode 100644 kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/TestIncrementalRows.java create mode 100644 kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/TestShutdownHook.java create mode 100644 kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/TestTableOutputFormat.java create mode 100644 kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/hs2connection/TestUserHS2ConnectionFileParser.java create mode 100644 kyuubi-hive-beeline/src/test/resources/DummyDriver.txt create mode 100644 kyuubi-hive-beeline/src/test/resources/hive-site.xml create mode 100644 kyuubi-hive-beeline/src/test/resources/test-hs2-conn-conf-kerberos-http.xml create mode 100644 kyuubi-hive-beeline/src/test/resources/test-hs2-conn-conf-kerberos-nossl.xml create mode 100644 kyuubi-hive-beeline/src/test/resources/test-hs2-conn-conf-kerberos-ssl.xml create mode 100644 kyuubi-hive-beeline/src/test/resources/test-hs2-connection-conf-list.xml create mode 100644 kyuubi-hive-beeline/src/test/resources/test-hs2-connection-config-noauth.xml create mode 100644 kyuubi-hive-beeline/src/test/resources/test-hs2-connection-multi-conf-list.xml create mode 100644 kyuubi-hive-beeline/src/test/resources/test-hs2-connection-zookeeper-config.xml diff --git a/.rat-excludes b/.rat-excludes index 6823fa44eb1..ba0b8f3ccc6 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -51,3 +51,5 @@ build/scala-*/** **/node_modules/** **/gen/* **/*.tokens +**/BeeLine.properties +**/sql-keywords.properties diff --git a/NOTICE-binary b/NOTICE-binary index b8104d39fad..eb8634f0fc0 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -189,9 +189,6 @@ JUnit (4.12) * License: Eclipse Public License -Hive Beeline -Copyright 2022 The Apache Software Foundation - Hive Common Copyright 2022 The Apache Software Foundation diff --git a/kyuubi-hive-beeline/pom.xml b/kyuubi-hive-beeline/pom.xml index 77337adcdba..33753ed9479 100644 --- a/kyuubi-hive-beeline/pom.xml +++ b/kyuubi-hive-beeline/pom.xml @@ -46,18 +46,6 @@ ${project.version} - - org.apache.hive - hive-beeline - ${hive.version} - - - * - * - - - - org.apache.hive hive-common @@ -229,6 +217,31 @@ ${skipTests} + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy + + copy + + process-test-resources + + + + org.postgresql + postgresql + ${postgresql.version} + true + ${project.build.directory} + + + + + + target/classes target/test-classes diff --git a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/AbstractCommandHandler.java b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/AbstractCommandHandler.java new file mode 100644 index 00000000000..2ceaf780dc0 --- /dev/null +++ b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/AbstractCommandHandler.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * This source file is based on code taken from SQLLine 1.0.2 + * See SQLLine notice in LICENSE + */ +package org.apache.hive.beeline; + +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import jline.console.completer.Completer; +import jline.console.completer.NullCompleter; + +/** An abstract implementation of CommandHandler. */ +public abstract class AbstractCommandHandler implements CommandHandler { + private final BeeLine beeLine; + private final String name; + private final String[] names; + private final String helpText; + private Completer[] parameterCompleters = new Completer[0]; + + protected transient Throwable lastException; + + public AbstractCommandHandler( + BeeLine beeLine, String[] names, String helpText, Completer[] completors) { + this.beeLine = beeLine; + name = names[0]; + this.names = names; + this.helpText = helpText; + if (completors == null || completors.length == 0) { + parameterCompleters = new Completer[] {new NullCompleter()}; + } else { + List c = new LinkedList(Arrays.asList(completors)); + c.add(new NullCompleter()); + parameterCompleters = c.toArray(new Completer[0]); + } + } + + @Override + public String getHelpText() { + return helpText; + } + + @Override + public String getName() { + return name; + } + + @Override + public String[] getNames() { + return names; + } + + @Override + public String matches(String line) { + if (line == null || line.length() == 0) { + return null; + } + + String[] parts = beeLine.split(line); + if (parts == null || parts.length == 0) { + return null; + } + + for (String name2 : names) { + if (name2.startsWith(parts[0])) { + return name2; + } + } + return null; + } + + public void setParameterCompleters(Completer[] parameterCompleters) { + this.parameterCompleters = parameterCompleters; + } + + @Override + public Completer[] getParameterCompleters() { + return parameterCompleters; + } + + @Override + public Throwable getLastException() { + return lastException; + } +} diff --git a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/AbstractOutputFormat.java b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/AbstractOutputFormat.java new file mode 100644 index 00000000000..138946dbec0 --- /dev/null +++ b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/AbstractOutputFormat.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * This source file is based on code taken from SQLLine 1.0.2 + * See SQLLine notice in LICENSE + */ +package org.apache.hive.beeline; + +/** Abstract OutputFormat. */ +abstract class AbstractOutputFormat implements OutputFormat { + + public int print(Rows rows) { + int count = 0; + Rows.Row header = (Rows.Row) rows.next(); + printHeader(header); + + while (rows.hasNext()) { + printRow(rows, header, (Rows.Row) rows.next()); + count++; + } + printFooter(header); + return count; + } + + abstract void printHeader(Rows.Row header); + + abstract void printFooter(Rows.Row header); + + abstract void printRow(Rows rows, Rows.Row header, Rows.Row row); +} diff --git a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/BeeLine.java b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/BeeLine.java new file mode 100644 index 00000000000..1706d1531d8 --- /dev/null +++ b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/BeeLine.java @@ -0,0 +1,2385 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * This source file is based on code taken from SQLLine 1.0.2 + * See SQLLine notice in LICENSE + */ +package org.apache.hive.beeline; + +import com.google.common.annotations.VisibleForTesting; +import java.io.BufferedReader; +import java.io.ByteArrayInputStream; +import java.io.Closeable; +import java.io.EOFException; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.PrintStream; +import java.io.SequenceInputStream; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.net.JarURLConnection; +import java.net.URL; +import java.net.URLConnection; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +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.ArrayList; +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.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; +import java.util.Properties; +import java.util.ResourceBundle; +import java.util.ServiceLoader; +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 jline.console.ConsoleReader; +import jline.console.completer.Completer; +import jline.console.completer.FileNameCompleter; +import jline.console.completer.StringsCompleter; +import jline.console.history.FileHistory; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.GnuParser; +import org.apache.commons.cli.OptionBuilder; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.io.IOUtils; +import org.apache.hive.beeline.cli.CliOptionsProcessor; +import org.apache.hive.beeline.hs2connection.BeelineConfFileParseException; +import org.apache.hive.beeline.hs2connection.BeelineSiteParseException; +import org.apache.hive.beeline.hs2connection.BeelineSiteParser; +import org.apache.hive.beeline.hs2connection.HS2ConnectionFileParser; +import org.apache.hive.beeline.hs2connection.HS2ConnectionFileUtils; +import org.apache.hive.beeline.hs2connection.HiveSiteHS2ConnectionFileParser; +import org.apache.hive.beeline.hs2connection.UserHS2ConnectionFileParser; +import org.apache.hive.common.util.ShutdownHookManager; +import org.apache.hive.jdbc.JdbcUriParseException; +import org.apache.hive.jdbc.Utils; +import org.apache.hive.jdbc.Utils.JdbcConnectionParams; +import org.apache.kyuubi.shaded.thrift.transport.TTransportException; + +/** + * 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 + *
+ */ +@SuppressWarnings("static-access") +public class BeeLine implements Closeable { + private static final ResourceBundle resourceBundle = + ResourceBundle.getBundle(BeeLine.class.getSimpleName()); + private final BeeLineSignalHandler signalHandler; + private final Runnable shutdownHook; + 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 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 InputStream inputStream = System.in; + private ConsoleReader consoleReader; + private List batch = null; + private final Reflector reflector = new Reflector(this); + private String dbName = null; + private String currentDatabase = null; + + private FileHistory history; + // Indicates if this instance of beeline is running in compatibility mode, or beeline mode + private boolean isBeeLine = true; + + // Indicates that we are in test mode. + // Print only the errors, the operation log and the query results. + private boolean isTestMode = false; + + private static final Options options = new Options(); + + public static final String BEELINE_DEFAULT_JDBC_DRIVER = "org.apache.hive.jdbc.HiveDriver"; + public static final String DEFAULT_DATABASE_NAME = "default"; + + private static final String SCRIPT_OUTPUT_PREFIX = ">>>"; + private static final int SCRIPT_OUTPUT_PAD_SIZE = 5; + + private static final int ERRNO_OK = 0; + private static final int ERRNO_ARGS = 1; + private static final int ERRNO_OTHER = 2; + + private static final String HIVE_VAR_PREFIX = "--hivevar"; + private static final String HIVE_CONF_PREFIX = "--hiveconf"; + private static final String PROP_FILE_PREFIX = "--property-file"; + static final String PASSWD_MASK = "[passwd stripped]"; + + private final Map formats = + map( + new Object[] { + "vertical", new VerticalOutputFormat(this), + "table", new TableOutputFormat(this), + "csv2", new SeparatedValuesOutputFormat(this, ','), + "tsv2", new SeparatedValuesOutputFormat(this, '\t'), + "dsv", new SeparatedValuesOutputFormat(this, BeeLineOpts.DEFAULT_DELIMITER_FOR_DSV), + "csv", new DeprecatedSeparatedValuesOutputFormat(this, ','), + "tsv", new DeprecatedSeparatedValuesOutputFormat(this, '\t'), + "xmlattr", new XMLAttributeOutputFormat(this), + "xmlelements", new XMLElementOutputFormat(this), + }); + + private List supportedLocalDriver = + new ArrayList(Arrays.asList("com.mysql.jdbc.Driver", "org.postgresql.Driver")); + + final CommandHandler[] commandHandlers = + new CommandHandler[] { + new ReflectiveCommandHandler(this, new String[] {"quit", "done", "exit"}, null), + new ReflectiveCommandHandler( + this, + new String[] {"connect", "open"}, + new Completer[] {new StringsCompleter(getConnectionURLExamples())}), + new ReflectiveCommandHandler( + this, new String[] {"describe"}, new Completer[] {new TableNameCompletor(this)}), + new ReflectiveCommandHandler( + this, new String[] {"indexes"}, new Completer[] {new TableNameCompletor(this)}), + new ReflectiveCommandHandler( + this, new String[] {"primarykeys"}, new Completer[] {new TableNameCompletor(this)}), + new ReflectiveCommandHandler( + this, new String[] {"exportedkeys"}, new Completer[] {new TableNameCompletor(this)}), + new ReflectiveCommandHandler(this, new String[] {"manual"}, null), + new ReflectiveCommandHandler( + this, new String[] {"importedkeys"}, new Completer[] {new TableNameCompletor(this)}), + new ReflectiveCommandHandler(this, new String[] {"procedures"}, null), + new ReflectiveCommandHandler(this, new String[] {"tables"}, null), + new ReflectiveCommandHandler(this, new String[] {"typeinfo"}, null), + new ReflectiveCommandHandler( + this, new String[] {"columns"}, new Completer[] {new TableNameCompletor(this)}), + new ReflectiveCommandHandler(this, new String[] {"reconnect"}, null), + new ReflectiveCommandHandler( + this, new String[] {"dropall"}, new Completer[] {new TableNameCompletor(this)}), + new ReflectiveCommandHandler(this, new String[] {"history"}, null), + new ReflectiveCommandHandler( + this, + new String[] {"metadata"}, + new Completer[] {new StringsCompleter(getMetadataMethodNames())}), + new ReflectiveCommandHandler(this, new String[] {"nativesql"}, null), + new ReflectiveCommandHandler(this, new String[] {"dbinfo"}, null), + new ReflectiveCommandHandler(this, new String[] {"rehash"}, null), + new ReflectiveCommandHandler(this, new String[] {"verbose"}, null), + new ReflectiveCommandHandler( + this, new String[] {"run"}, new Completer[] {new FileNameCompleter()}), + new ReflectiveCommandHandler(this, new String[] {"batch"}, null), + new ReflectiveCommandHandler(this, new String[] {"list"}, null), + new ReflectiveCommandHandler(this, new String[] {"all"}, null), + new ReflectiveCommandHandler(this, new String[] {"go", "#"}, null), + new ReflectiveCommandHandler( + this, new String[] {"script"}, new Completer[] {new FileNameCompleter()}), + new ReflectiveCommandHandler( + this, new String[] {"record"}, new Completer[] {new FileNameCompleter()}), + new ReflectiveCommandHandler(this, new String[] {"brief"}, null), + new ReflectiveCommandHandler(this, new String[] {"close"}, null), + new ReflectiveCommandHandler(this, new String[] {"closeall"}, null), + new ReflectiveCommandHandler( + this, + new String[] {"isolation"}, + new Completer[] {new StringsCompleter(getIsolationLevels())}), + new ReflectiveCommandHandler( + this, + new String[] {"outputformat"}, + new Completer[] {new StringsCompleter(formats.keySet().toArray(new String[0]))}), + new ReflectiveCommandHandler(this, new String[] {"autocommit"}, null), + new ReflectiveCommandHandler(this, new String[] {"commit"}, null), + new ReflectiveCommandHandler( + this, new String[] {"properties"}, new Completer[] {new FileNameCompleter()}), + new ReflectiveCommandHandler(this, new String[] {"rollback"}, null), + new ReflectiveCommandHandler(this, new String[] {"help", "?"}, null), + new ReflectiveCommandHandler(this, new String[] {"set"}, getOpts().optionCompleters()), + new ReflectiveCommandHandler(this, new String[] {"save"}, null), + new ReflectiveCommandHandler(this, new String[] {"scan"}, null), + new ReflectiveCommandHandler(this, new String[] {"sql"}, null), + new ReflectiveCommandHandler(this, new String[] {"sh"}, null), + new ReflectiveCommandHandler(this, new String[] {"call"}, null), + new ReflectiveCommandHandler( + this, new String[] {"nullemptystring"}, new Completer[] {new BooleanCompleter()}), + new ReflectiveCommandHandler(this, new String[] {"addlocaldriverjar"}, null), + new ReflectiveCommandHandler(this, new String[] {"addlocaldrivername"}, null), + new ReflectiveCommandHandler(this, new String[] {"delimiter"}, null) + }; + + private final Completer beeLineCommandCompleter = + new BeeLineCommandCompleter(Arrays.asList(commandHandlers)); + + 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.console.ConsoleReader"); + } catch (Throwable t) { + throw new ExceptionInInitializerError("jline-missing"); + } + } + + static { + // -d + options.addOption( + OptionBuilder.hasArg() + .withArgName("driver class") + .withDescription("The driver class to use") + .create('d')); + + // -u + options.addOption( + OptionBuilder.hasArg() + .withArgName("database url") + .withDescription("The JDBC URL to connect to") + .create('u')); + + // -c + options.addOption( + OptionBuilder.hasArg() + .withArgName("named JDBC URL in beeline-site.xml") + .withDescription( + "The named JDBC URL to connect to, which should be present in " + + "beeline-site.xml as the value of beeline.hs2.jdbc.url.") + .create('c')); + + // -r + options.addOption( + OptionBuilder.withLongOpt("reconnect") + .withDescription("Reconnect to last saved connect url (in conjunction with !save)") + .create('r')); + + // -n + options.addOption( + OptionBuilder.hasArg() + .withArgName("username") + .withDescription("The username to connect as") + .create('n')); + + // -p + options.addOption( + OptionBuilder.hasArg() + .withArgName("password") + .withDescription("The password to connect as") + .hasOptionalArg() + .create('p')); + + // -w (or) --password-file + options.addOption( + OptionBuilder.hasArg() + .withArgName("password-file") + .withDescription("The password file to read password from") + .withLongOpt("password-file") + .create('w')); + + // -a + options.addOption( + OptionBuilder.hasArg() + .withArgName("authType") + .withDescription("The authentication type") + .create('a')); + + // -i + options.addOption( + OptionBuilder.hasArgs() + .withArgName("init") + .withDescription("The script file for initialization") + .create('i')); + + // -e + options.addOption( + OptionBuilder.hasArgs() + .withArgName("query") + .withDescription("The query that should be executed") + .create('e')); + + // -f