Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

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

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
9 changes: 0 additions & 9 deletions NOTICE-binary
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down
65 changes: 8 additions & 57 deletions kyuubi-hive-beeline/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -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>
Expand All @@ -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>
Expand All @@ -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>
Expand Down Expand Up @@ -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>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;

/**
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -1226,7 +1226,7 @@ private int executeFile(String fileName) {
handleException(t);
return ERRNO_OTHER;
} finally {
IOUtils.closeStream(fileStream);
IOUtils.closeQuietly(fileStream);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {

Expand Down Expand Up @@ -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();
Expand Down Expand Up @@ -1289,11 +1290,11 @@ private void addCmdPart(List<String> 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() {
Expand All @@ -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;
Expand All @@ -1330,7 +1331,7 @@ static class LogRunnable implements Runnable {

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);
Expand Down Expand Up @@ -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<String> logs = null;
do {
try {
logs = hiveStatement.getQueryLog();
logs = hiveStatement.getExecLog();
} catch (SQLException e) {
beeLine.error(new SQLWarning(e));
return;
Expand Down Expand Up @@ -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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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:";
Expand Down Expand Up @@ -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();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -40,7 +41,7 @@
* <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";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we still support org.apache.hive.jdbc.HiveDriver right ?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we have no such guarantee, the internally used JDBC driver is not user-faced stuff, we only need to ensure that user could smoothly migrate from Hive Beeline to Kyuubi Beeline without changing the JDBC URL.

private static final String BEELINE_EXIT = "beeline.system.exit";
private static Connection con = null;
private static boolean noClose = false;
Expand Down Expand Up @@ -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);
}
Expand Down Expand Up @@ -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());
Expand Down Expand Up @@ -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<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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down Expand Up @@ -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 =
Expand Down
Loading
Loading