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.hivehive-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/classestarget/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