diff --git a/NOTICE-binary b/NOTICE-binary
index eb8634f0fc0..a1b304dfaa9 100644
--- a/NOTICE-binary
+++ b/NOTICE-binary
@@ -192,15 +192,6 @@ JUnit (4.12)
Hive Common
Copyright 2022 The Apache Software Foundation
-Hive JDBC
-Copyright 2022 The Apache Software Foundation
-
-Hive Service
-Copyright 2022 The Apache Software Foundation
-
-Hive Service RPC
-Copyright 2022 The Apache Software Foundation
-
Apache HttpClient
Copyright 1999-2020 The Apache Software Foundation
diff --git a/kyuubi-hive-beeline/pom.xml b/kyuubi-hive-beeline/pom.xml
index 33753ed9479..0b3bbf48f6d 100644
--- a/kyuubi-hive-beeline/pom.xml
+++ b/kyuubi-hive-beeline/pom.xml
@@ -59,44 +59,13 @@
- org.apache.hive
- hive-jdbc
- ${hive.version}
-
-
- *
- *
-
-
-
-
-
- org.apache.hive
- hive-service-rpc
- ${hive.version}
-
-
- *
- *
-
-
-
-
-
- org.apache.hive
- hive-service
- ${hive.version}
-
-
- *
- *
-
-
+ org.apache.hadoop
+ hadoop-client-api
org.apache.hadoop
- hadoop-client-api
+ hadoop-client-runtime
@@ -110,14 +79,8 @@
- org.mockito
- mockito-core
- test
-
-
-
- commons-lang
- commons-lang
+ org.apache.commons
+ commons-lang3
@@ -128,13 +91,6 @@
net.sf.supercsv
super-csv
- ${hive.client.supercsv.version}
-
-
- *
- *
-
-
@@ -163,14 +119,9 @@
- org.apache.hadoop
- hadoop-client-runtime
-
-
- *
- *
-
-
+ org.mockito
+ mockito-core
+ test
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
index 1706d1531d8..e3b29206c05 100644
--- 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
@@ -86,10 +86,10 @@
import org.apache.commons.cli.OptionBuilder;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
+import org.apache.commons.io.IOUtils;
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;
@@ -99,9 +99,9 @@
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.jdbc.hive.JdbcConnectionParams;
+import org.apache.kyuubi.jdbc.hive.JdbcUriParseException;
+import org.apache.kyuubi.jdbc.hive.Utils;
import org.apache.kyuubi.shaded.thrift.transport.TTransportException;
/**
@@ -1178,7 +1178,7 @@ int runInit() {
}
private int embeddedConnect() {
- if (!execCommandWithPrefix("!connect " + Utils.URL_PREFIX + " '' ''")) {
+ if (!execCommandWithPrefix("!connect jdbc:hive2:// '' ''")) {
return ERRNO_OTHER;
} else {
return ERRNO_OK;
@@ -1226,7 +1226,7 @@ private int executeFile(String fileName) {
handleException(t);
return ERRNO_OTHER;
} finally {
- IOUtils.closeStream(fileStream);
+ IOUtils.closeQuietly(fileStream);
}
}
diff --git a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/ClientCommandHookFactory.java b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/ClientCommandHookFactory.java
index 935f1cfbff0..cebcad1fed0 100644
--- a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/ClientCommandHookFactory.java
+++ b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/ClientCommandHookFactory.java
@@ -18,7 +18,7 @@
package org.apache.hive.beeline;
import java.sql.SQLException;
-import org.apache.hive.jdbc.Utils;
+import org.apache.kyuubi.jdbc.hive.Utils;
/** We need to update some client side information after executing some Hive Commands */
public class ClientCommandHookFactory {
diff --git a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/Commands.java b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/Commands.java
index 6c59b0401c9..a1df5fb5500 100644
--- a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/Commands.java
+++ b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/Commands.java
@@ -58,11 +58,12 @@
import org.apache.hadoop.hive.conf.SystemVariables;
import org.apache.hadoop.hive.conf.VariableSubstitution;
import org.apache.hadoop.io.IOUtils;
+import org.apache.hive.beeline.logs.KyuubiBeelineInPlaceUpdateStream;
import org.apache.hive.common.util.HiveStringUtils;
-import org.apache.hive.jdbc.HiveStatement;
-import org.apache.hive.jdbc.Utils;
-import org.apache.hive.jdbc.Utils.JdbcConnectionParams;
-import org.apache.hive.jdbc.logs.InPlaceUpdateStream;
+import org.apache.kyuubi.jdbc.hive.JdbcConnectionParams;
+import org.apache.kyuubi.jdbc.hive.KyuubiStatement;
+import org.apache.kyuubi.jdbc.hive.Utils;
+import org.apache.kyuubi.jdbc.hive.logs.InPlaceUpdateStream;
public class Commands {
@@ -986,10 +987,10 @@ private boolean executeInternal(String sql, boolean call) {
logThread = new Thread(createLogRunnable(stmnt, eventNotifier));
logThread.setDaemon(true);
logThread.start();
- if (stmnt instanceof HiveStatement) {
- // HiveStatement hiveStatement = (HiveStatement) stmnt;
- // hiveStatement.setInPlaceUpdateStream(
- // new BeelineInPlaceUpdateStream(beeLine.getErrorStream(), eventNotifier));
+ if (stmnt instanceof KyuubiStatement) {
+ KyuubiStatement hiveStatement = (KyuubiStatement) stmnt;
+ hiveStatement.setInPlaceUpdateStream(
+ new KyuubiBeelineInPlaceUpdateStream(beeLine.getErrorStream(), eventNotifier));
}
hasResults = stmnt.execute(sql);
logThread.interrupt();
@@ -1289,11 +1290,11 @@ private void addCmdPart(List cmdList, StringBuilder command, String cmdp
private Runnable createLogRunnable(
final Statement statement, InPlaceUpdateStream.EventNotifier eventNotifier) {
- if (statement instanceof HiveStatement) {
+ if (statement instanceof KyuubiStatement) {
return new LogRunnable(
- this, (HiveStatement) statement, DEFAULT_QUERY_PROGRESS_INTERVAL, eventNotifier);
+ this, (KyuubiStatement) statement, DEFAULT_QUERY_PROGRESS_INTERVAL, eventNotifier);
} else {
- beeLine.debug("The statement instance is not HiveStatement type: " + statement.getClass());
+ beeLine.debug("The statement instance is not KyuubiStatement type: " + statement.getClass());
return new Runnable() {
@Override
public void run() {
@@ -1313,13 +1314,13 @@ private void debug(String message) {
static class LogRunnable implements Runnable {
private final Commands commands;
- private final HiveStatement hiveStatement;
+ private final KyuubiStatement hiveStatement;
private final long queryProgressInterval;
private final InPlaceUpdateStream.EventNotifier notifier;
LogRunnable(
Commands commands,
- HiveStatement hiveStatement,
+ KyuubiStatement hiveStatement,
long queryProgressInterval,
InPlaceUpdateStream.EventNotifier eventNotifier) {
this.hiveStatement = hiveStatement;
@@ -1330,7 +1331,7 @@ static class LogRunnable implements Runnable {
private void updateQueryLog() {
try {
- List queryLogs = hiveStatement.getQueryLog();
+ List queryLogs = hiveStatement.getExecLog();
for (String log : queryLogs) {
if (!commands.beeLine.isTestMode()) {
commands.beeLine.info(log);
@@ -1371,12 +1372,12 @@ public void run() {
}
private void showRemainingLogsIfAny(Statement statement) {
- if (statement instanceof HiveStatement) {
- HiveStatement hiveStatement = (HiveStatement) statement;
+ if (statement instanceof KyuubiStatement) {
+ KyuubiStatement hiveStatement = (KyuubiStatement) statement;
List logs = null;
do {
try {
- logs = hiveStatement.getQueryLog();
+ logs = hiveStatement.getExecLog();
} catch (SQLException e) {
beeLine.error(new SQLWarning(e));
return;
@@ -1629,9 +1630,7 @@ public boolean connect(Properties props) throws IOException {
props.setProperty(JdbcConnectionParams.AUTH_USER, username);
if (password == null) {
password =
- beeLine
- .getConsoleReader()
- .readLine("Enter password for " + urlForPrompt + ": ", new Character('*'));
+ beeLine.getConsoleReader().readLine("Enter password for " + urlForPrompt + ": ", '*');
}
props.setProperty(JdbcConnectionParams.AUTH_PASSWD, password);
}
diff --git a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/DatabaseConnection.java b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/DatabaseConnection.java
index a57ae6331c3..6247155971b 100644
--- a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/DatabaseConnection.java
+++ b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/DatabaseConnection.java
@@ -38,7 +38,7 @@
import java.util.TreeSet;
import jline.console.completer.ArgumentCompleter;
import jline.console.completer.Completer;
-import org.apache.hive.jdbc.HiveConnection;
+import org.apache.kyuubi.jdbc.hive.KyuubiConnection;
class DatabaseConnection {
private static final String HIVE_VAR_PREFIX = "hivevar:";
@@ -272,8 +272,8 @@ String getUrl() {
}
public String getConnectedUrl() {
- if (connection instanceof HiveConnection) {
- return ((HiveConnection) connection).getConnectedUrl();
+ if (connection instanceof KyuubiConnection) {
+ return ((KyuubiConnection) connection).getConnectedUrl();
}
return getUrl();
}
diff --git a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java
index d721f48226f..4a93db9f8b4 100644
--- a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java
+++ b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/SeparatedValuesOutputFormat.java
@@ -23,8 +23,8 @@
package org.apache.hive.beeline;
import org.apache.commons.io.output.StringBuilderWriter;
-import org.apache.commons.lang.BooleanUtils;
-import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.BooleanUtils;
+import org.apache.commons.lang3.StringUtils;
import org.supercsv.encoder.CsvEncoder;
import org.supercsv.encoder.DefaultCsvEncoder;
import org.supercsv.encoder.SelectiveCsvEncoder;
diff --git a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileUtils.java b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileUtils.java
index f4476795568..c21f6120a3b 100644
--- a/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileUtils.java
+++ b/kyuubi-hive-beeline/src/main/java/org/apache/hive/beeline/hs2connection/HS2ConnectionFileUtils.java
@@ -23,7 +23,7 @@
import java.util.List;
import java.util.Map;
import java.util.Properties;
-import org.apache.hive.jdbc.Utils.JdbcConnectionParams;
+import org.apache.kyuubi.jdbc.hive.JdbcConnectionParams;
public class HS2ConnectionFileUtils {
diff --git a/kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/ProxyAuthTest.java b/kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/ProxyAuthTest.java
index 56595e83f2b..c4e9ae8e7dc 100644
--- a/kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/ProxyAuthTest.java
+++ b/kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/ProxyAuthTest.java
@@ -26,10 +26,11 @@
import java.sql.Statement;
import javax.security.auth.login.LoginException;
import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hive.jdbc.HiveConnection;
-import org.apache.hive.service.auth.HiveAuthConstants;
-import org.apache.hive.service.cli.session.SessionUtils;
+import org.apache.hadoop.security.token.Token;
+import org.apache.kyuubi.jdbc.hive.KyuubiConnection;
/**
* Simple client application to test various direct and proxy connection to HiveServer2 Note that
@@ -40,7 +41,7 @@
*
*/
public class ProxyAuthTest {
- private static final String driverName = "org.apache.hive.jdbc.HiveDriver";
+ private static final String driverName = "org.apache.kyuubi.jdbc.KyuubiHiveDriver";
private static final String BEELINE_EXIT = "beeline.system.exit";
private static Connection con = null;
private static boolean noClose = false;
@@ -86,7 +87,7 @@ public static void main(String[] args) throws Exception {
con = DriverManager.getConnection(url);
System.out.println("Connected successfully to " + url);
// get delegation token for the given proxy user
- String token = ((HiveConnection) con).getDelegationToken(proxyUser, serverPrincipal);
+ String token = ((KyuubiConnection) con).getDelegationToken(proxyUser, serverPrincipal);
if ("true".equals(System.getProperty("proxyAuth.debug", "false"))) {
System.out.println("Got token: " + token);
}
@@ -190,7 +191,7 @@ public static void main(String[] args) throws Exception {
System.out.println("Connected successfully to " + url);
runTest();
- ((HiveConnection) con).cancelDelegationToken(token);
+ ((KyuubiConnection) con).cancelDelegationToken(token);
con.close();
} catch (SQLException e) {
System.out.println("*** SQLException: " + e.getMessage() + " : " + e.getSQLState());
@@ -222,11 +223,27 @@ private static UserGroupInformation getUGI() throws LoginException, IOException
return UserGroupInformation.getCurrentUser();
}
+ private static final String HS2_CLIENT_TOKEN = "hiveserver2ClientToken";
+
private static void storeTokenInJobConf(String tokenStr) throws Exception {
- SessionUtils.setTokenStr(getUGI(), tokenStr, HiveAuthConstants.HS2_CLIENT_TOKEN);
+ setTokenStr(getUGI(), tokenStr, HS2_CLIENT_TOKEN);
System.out.println("Stored token " + tokenStr);
}
+ private static void setTokenStr(UserGroupInformation ugi, String tokenStr, String tokenService)
+ throws IOException {
+ Token delegationToken = createToken(tokenStr, tokenService);
+ ugi.addToken(delegationToken);
+ }
+
+ private static Token createToken(String tokenStr, String tokenService)
+ throws IOException {
+ Token delegationToken = new Token<>();
+ delegationToken.decodeFromUrlString(tokenStr);
+ delegationToken.setService(new Text(tokenService));
+ return delegationToken;
+ }
+
// run sql operations
private static void runTest() throws Exception {
// craete table and check dir ownership
diff --git a/kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/TestClientCommandHookFactory.java b/kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/TestClientCommandHookFactory.java
index a8cd480f125..6aaef122206 100644
--- a/kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/TestClientCommandHookFactory.java
+++ b/kyuubi-hive-beeline/src/test/java/org/apache/hive/beeline/TestClientCommandHookFactory.java
@@ -24,7 +24,6 @@
import java.sql.Connection;
import java.sql.SQLException;
import junit.framework.Assert;
-import org.junit.Ignore;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
@@ -103,7 +102,6 @@ public void testUseHook() {
}
@Test
- @Ignore("Fix later: NoClassDefFound org/apache/curator/RetryPolicy")
public void testConnectHook() {
BeeLine beeLine = setupMockData(true, true);
ClientHook hook =
diff --git a/pom.xml b/pom.xml
index 8f3c0baaf20..85b2e023225 100644
--- a/pom.xml
+++ b/pom.xml
@@ -207,6 +207,7 @@
${apache.archive.dist}/spark/spark-${spark.version}
false
3.42.0.0
+ 2.2.0
2.2.1
0.41.0
@@ -222,9 +223,6 @@
v18.16.0
v8.6.1
-
- 2.2.0
-
kyuubi-hive-jdbc
UTF-8
@@ -1259,6 +1257,12 @@
${fliptables.verion}
+
+ net.sf.supercsv
+ super-csv
+ ${supercsv.version}
+
+
org.glassfish.jersey.core