[KYUUBI #6153] Cut out Hive JDBC and Hive Service deps in beeline module

# 🔍 Description
## Issue References 🔗

This is the next step of https://github.com/apache/kyuubi/issues/6146, cutting out most Hive deps(excepting `hive-common`) and recovering the skipped tests via minor code tunning.

## Describe Your Solution 🔧

- Drop `hive-jdbc`, `hive-service`, `hive-service-rpc` deps in the beeline module.
- Migrate from `commons-lang` to `commons-lang3` in the beeline module.
- Recover the skipped test `TestClientCommandHookFactory#connectHook`

## Types of changes 🔖

- [x] 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 🧪

Pass GA, and manually test to ensure the following error has gone.

Before
```
roothadoop-master1:/opt/kyuubi# bin/beeline --version
Warn: Not find kyuubi environment file /etc/kyuubi/conf/kyuubi-env.sh, using default ones...
java.lang.NoClassDefFoundError: org/apache/curator/RetryPolicy
```

After
```
roothadoop-master1:/opt/kyuubi# bin/beeline --version
Connecting to jdbc:hive2://hadoop-master1.orb.local:10000/default;password=hive;user=hive
Connected to: Apache Hive (version 2.3.9)
Driver: Kyuubi Project Hive JDBC Client (version 1.9.0-SNAPSHOT)
Beeline version 1.9.0-SNAPSHOT by Apache Kyuubi
0: jdbc:hive2://hadoop-master1.orb.local:1000>
```

---

# 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 #6153 from pan3793/beeline-2.

Closes #6153

8cd52e509 [Cheng Pan] notice
d03c72992 [Cheng Pan] minor
5d16bf4ce [Cheng Pan] beeline test pass

Authored-by: Cheng Pan <chengpan@apache.org>
Signed-off-by: Cheng Pan <chengpan@apache.org>
This commit is contained in:
Cheng Pan 2024-03-11 15:21:14 +08:00
parent 1dddd08938
commit 9fd392b7e9
No known key found for this signature in database
GPG Key ID: 8001952629BCC75D
11 changed files with 71 additions and 111 deletions

View File

@ -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

View File

@ -59,44 +59,13 @@
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-jdbc</artifactId>
<version>${hive.version}</version>
<exclusions>
<exclusion>
<groupId>*</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-service-rpc</artifactId>
<version>${hive.version}</version>
<exclusions>
<exclusion>
<groupId>*</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-service</artifactId>
<version>${hive.version}</version>
<exclusions>
<exclusion>
<groupId>*</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client-api</artifactId>
<artifactId>hadoop-client-runtime</artifactId>
</dependency>
<dependency>
@ -110,14 +79,8 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>commons-lang</groupId>
<artifactId>commons-lang</artifactId>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
<dependency>
@ -128,13 +91,6 @@
<dependency>
<groupId>net.sf.supercsv</groupId>
<artifactId>super-csv</artifactId>
<version>${hive.client.supercsv.version}</version>
<exclusions>
<exclusion>
<groupId>*</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
@ -163,14 +119,9 @@
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client-runtime</artifactId>
<exclusions>
<exclusion>
<groupId>*</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
<groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>

View File

@ -86,10 +86,10 @@ 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.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.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.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 @@ public class BeeLine implements Closeable {
}
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 @@ public class BeeLine implements Closeable {
handleException(t);
return ERRNO_OTHER;
} finally {
IOUtils.closeStream(fileStream);
IOUtils.closeQuietly(fileStream);
}
}

View File

@ -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 {

View File

@ -58,11 +58,12 @@ import org.apache.hadoop.hive.conf.HiveVariableSource;
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 @@ public class Commands {
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 @@ public class Commands {
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 @@ public class Commands {
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 @@ public class Commands {
private void updateQueryLog() {
try {
List<String> queryLogs = hiveStatement.getQueryLog();
List<String> queryLogs = hiveStatement.getExecLog();
for (String log : queryLogs) {
if (!commands.beeLine.isTestMode()) {
commands.beeLine.info(log);
@ -1371,12 +1372,12 @@ public class Commands {
}
private void showRemainingLogsIfAny(Statement statement) {
if (statement instanceof HiveStatement) {
HiveStatement hiveStatement = (HiveStatement) statement;
if (statement instanceof KyuubiStatement) {
KyuubiStatement hiveStatement = (KyuubiStatement) statement;
List<String> logs = null;
do {
try {
logs = hiveStatement.getQueryLog();
logs = hiveStatement.getExecLog();
} catch (SQLException e) {
beeLine.error(new SQLWarning(e));
return;
@ -1629,9 +1630,7 @@ public class Commands {
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);
}

View File

@ -38,7 +38,7 @@ import java.util.Set;
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 @@ class DatabaseConnection {
}
public String getConnectedUrl() {
if (connection instanceof HiveConnection) {
return ((HiveConnection) connection).getConnectedUrl();
if (connection instanceof KyuubiConnection) {
return ((KyuubiConnection) connection).getConnectedUrl();
}
return getUrl();
}

View File

@ -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;

View File

@ -23,7 +23,7 @@ import java.util.Collections;
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 {

View File

@ -26,10 +26,11 @@ import java.sql.SQLException;
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 @@ import org.apache.hive.service.cli.session.SessionUtils;
* <client-principal>
*/
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 class ProxyAuthTest {
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 class ProxyAuthTest {
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 @@ public class ProxyAuthTest {
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<DelegationTokenIdentifier> delegationToken = createToken(tokenStr, tokenService);
ugi.addToken(delegationToken);
}
private static Token<DelegationTokenIdentifier> createToken(String tokenStr, String tokenService)
throws IOException {
Token<DelegationTokenIdentifier> 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

View File

@ -24,7 +24,6 @@ import static org.mockito.Mockito.when;
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 class TestClientCommandHookFactory {
}
@Test
@Ignore("Fix later: NoClassDefFound org/apache/curator/RetryPolicy")
public void testConnectHook() {
BeeLine beeLine = setupMockData(true, true);
ClientHook hook =

10
pom.xml
View File

@ -207,6 +207,7 @@
<spark.archive.mirror>${apache.archive.dist}/spark/spark-${spark.version}</spark.archive.mirror>
<spark.archive.download.skip>false</spark.archive.download.skip>
<sqlite.version>3.42.0.0</sqlite.version>
<supercsv.version>2.2.0</supercsv.version>
<swagger.version>2.2.1</swagger.version>
<testcontainers-scala.version>0.41.0</testcontainers-scala.version>
<!-- https://github.com/ThreeTen/threeten-extra/issues/226 -->
@ -222,9 +223,6 @@
<node.version>v18.16.0</node.version>
<pnpm.version>v8.6.1</pnpm.version>
<!-- apply to kyuubi-hive-jdbc/kyuubi-hive-beeline module -->
<hive.client.supercsv.version>2.2.0</hive.client.supercsv.version>
<hive.jdbc.artifact>kyuubi-hive-jdbc</hive.jdbc.artifact>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
@ -1259,6 +1257,12 @@
<version>${fliptables.verion}</version>
</dependency>
<dependency>
<groupId>net.sf.supercsv</groupId>
<artifactId>super-csv</artifactId>
<version>${supercsv.version}</version>
</dependency>
<!-- RESTful service required -->
<dependency>
<groupId>org.glassfish.jersey.core</groupId>