You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by ch...@apache.org on 2021/08/02 09:35:09 UTC

[incubator-kyuubi] branch master updated: [KYUUBI #884] Rewrite kyuubi-hive-jdbc in Java

This is an automated email from the ASF dual-hosted git repository.

chengpan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-kyuubi.git


The following commit(s) were added to refs/heads/master by this push:
     new 1a8b4eb  [KYUUBI #884] Rewrite kyuubi-hive-jdbc in Java
1a8b4eb is described below

commit 1a8b4ebaaeb8308018cab0c65497c74bc38f4a85
Author: Cheng Pan <ch...@apache.org>
AuthorDate: Mon Aug 2 17:35:00 2021 +0800

    [KYUUBI #884] Rewrite kyuubi-hive-jdbc in Java
    
    <!--
    Thanks for sending a pull request!
    
    Here are some tips for you:
      1. If this is your first time, please read our contributor guidelines: https://kyuubi.readthedocs.io/en/latest/community/contributions.html
      2. If the PR is related to an issue in https://github.com/apache/incubator-kyuubi/issues, add '[KYUUBI #XXXX]' in your PR title, e.g., '[KYUUBI #XXXX] Your PR title ...'.
      3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][KYUUBI #XXXX] Your PR title ...'.
    -->
    
    ### _Why are the changes needed?_
    <!--
    Please clarify why the changes are needed. For instance,
      1. If you add a feature, you can talk about the use case of it.
      2. If you fix a bug, you can clarify why it is a bug.
    -->
    close #765
    
    ### _How was this patch tested?_
    - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible
    
    - [ ] Add screenshots for manual tests if appropriate
    
    - [ ] [Run test](https://kyuubi.readthedocs.io/en/latest/tools/testing.html#running-tests) locally before make a pull request
    
    Closes #884 from pan3793/jdbc-java.
    
    Closes #884
    
    ca7bad48 [Cheng Pan] Rewrite kyuubi-hive-jdbc in Java
    
    Authored-by: Cheng Pan <ch...@apache.org>
    Signed-off-by: Cheng Pan <ch...@apache.org>
---
 dev/kyuubi-extension-spark_3.1/pom.xml             |  6 ++
 dev/kyuubi-tpcds/pom.xml                           |  6 ++
 kyuubi-common/pom.xml                              |  4 ++
 .../org/apache/kyuubi/jdbc/KyuubiConnection.java   | 52 ++++++++++++++++
 .../apache/kyuubi/jdbc/KyuubiDatabaseMetaData.java | 70 ++++++++++++++++++++++
 .../org/apache/kyuubi/jdbc/KyuubiDriver.java}      | 35 +++++------
 .../org/apache/kyuubi/jdbc/KyuubiConnection.scala  | 41 -------------
 .../kyuubi/jdbc/KyuubiDatabaseMetaData.scala       | 59 ------------------
 pom.xml                                            |  5 --
 9 files changed, 156 insertions(+), 122 deletions(-)

diff --git a/dev/kyuubi-extension-spark_3.1/pom.xml b/dev/kyuubi-extension-spark_3.1/pom.xml
index 95e4c22..3dd5aea 100644
--- a/dev/kyuubi-extension-spark_3.1/pom.xml
+++ b/dev/kyuubi-extension-spark_3.1/pom.xml
@@ -33,6 +33,12 @@
 
     <dependencies>
         <dependency>
+            <groupId>org.scala-lang</groupId>
+            <artifactId>scala-library</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
             <groupId>org.apache.spark</groupId>
             <artifactId>spark-sql_${scala.binary.version}</artifactId>
             <scope>provided</scope>
diff --git a/dev/kyuubi-tpcds/pom.xml b/dev/kyuubi-tpcds/pom.xml
index 7eb5b83..77f41f4 100644
--- a/dev/kyuubi-tpcds/pom.xml
+++ b/dev/kyuubi-tpcds/pom.xml
@@ -33,6 +33,12 @@
 
     <dependencies>
         <dependency>
+            <groupId>org.scala-lang</groupId>
+            <artifactId>scala-library</artifactId>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
             <groupId>org.apache.spark</groupId>
             <artifactId>spark-sql_${scala.binary.version}</artifactId>
             <scope>provided</scope>
diff --git a/kyuubi-common/pom.xml b/kyuubi-common/pom.xml
index 53596ce..48aed51 100644
--- a/kyuubi-common/pom.xml
+++ b/kyuubi-common/pom.xml
@@ -32,6 +32,10 @@
     <name>Kyuubi Project Common</name>
 
     <dependencies>
+        <dependency>
+            <groupId>org.scala-lang</groupId>
+            <artifactId>scala-library</artifactId>
+        </dependency>
 
         <dependency>
             <groupId>org.slf4j</groupId>
diff --git a/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/KyuubiConnection.java b/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/KyuubiConnection.java
new file mode 100644
index 0000000..c7a832a
--- /dev/null
+++ b/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/KyuubiConnection.java
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+
+package org.apache.kyuubi.jdbc;
+
+import java.lang.reflect.Field;
+import java.sql.DatabaseMetaData;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import org.apache.hive.jdbc.HiveConnection;
+import org.apache.hive.service.rpc.thrift.TCLIService;
+import org.apache.hive.service.rpc.thrift.TSessionHandle;
+
+public class KyuubiConnection extends HiveConnection {
+
+    public KyuubiConnection(String url, Properties info) throws SQLException {
+        super(url, info);
+    }
+
+    @Override
+    public DatabaseMetaData getMetaData() throws SQLException {
+        if (isClosed()) {
+            throw new SQLException("Connection is closed");
+        }
+        try {
+            Field clientField = HiveConnection.class.getDeclaredField("client");
+            clientField.setAccessible(true);
+            TCLIService.Iface client = (TCLIService.Iface) clientField.get(this);
+            Field handleField = HiveConnection.class.getDeclaredField("sessHandle");
+            handleField.setAccessible(true);
+            TSessionHandle sessionHandle = (TSessionHandle) handleField.get(this);
+            return new KyuubiDatabaseMetaData(this, client, sessionHandle);
+        } catch (NoSuchFieldException | IllegalAccessException rethrow) {
+            throw new RuntimeException(rethrow);
+        }
+    }
+}
diff --git a/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/KyuubiDatabaseMetaData.java b/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/KyuubiDatabaseMetaData.java
new file mode 100644
index 0000000..dc4aeff
--- /dev/null
+++ b/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/KyuubiDatabaseMetaData.java
@@ -0,0 +1,70 @@
+/*
+ * 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.
+ */
+
+package org.apache.kyuubi.jdbc;
+
+import org.apache.hive.jdbc.HiveDatabaseMetaData;
+import org.apache.hive.jdbc.HiveQueryResultSet;
+import org.apache.hive.service.cli.HiveSQLException;
+import org.apache.hive.service.rpc.thrift.*;
+import org.apache.thrift.TException;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+
+public class KyuubiDatabaseMetaData extends HiveDatabaseMetaData {
+    private final KyuubiConnection conn;
+    private final TCLIService.Iface client;
+    private final TSessionHandle sessHandle;
+
+    public KyuubiDatabaseMetaData(KyuubiConnection conn, TCLIService.Iface client, TSessionHandle sessHandle) {
+        super(conn, client, sessHandle);
+        this.conn = conn;
+        this.client = client;
+        this.sessHandle = sessHandle;
+    }
+
+    @Override
+    public ResultSet getTables(String catalog, String schemaPattern,
+                               String tableNamePattern, String[] types) throws SQLException {
+
+        TGetTablesReq getTableReq = new TGetTablesReq(sessHandle);
+        getTableReq.setCatalogName(catalog);
+        getTableReq.setSchemaName(schemaPattern == null ? "%" : schemaPattern);
+        getTableReq.setTableName(tableNamePattern);
+        if (types != null) {
+            getTableReq.setTableTypes(Arrays.asList(types));
+        }
+        TGetTablesResp getTableResp;
+        try {
+            getTableResp = client.GetTables(getTableReq);
+        } catch (TException rethrow) {
+            throw new SQLException(rethrow.getMessage(), "08S01", rethrow);
+        }
+        TStatus tStatus = getTableResp.getStatus();
+        if (tStatus.getStatusCode() != TStatusCode.SUCCESS_STATUS) {
+            throw new HiveSQLException(tStatus);
+        }
+        new HiveQueryResultSet.Builder(conn)
+                .setClient(client)
+                .setSessionHandle(sessHandle)
+                .setStmtHandle(getTableResp.getOperationHandle())
+                .build();
+        return super.getTables(catalog, schemaPattern, tableNamePattern, types);
+    }
+}
diff --git a/kyuubi-hive-jdbc/src/main/scala/org/apache/kyuubi/jdbc/KyuubiDriver.scala b/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/KyuubiDriver.java
similarity index 56%
rename from kyuubi-hive-jdbc/src/main/scala/org/apache/kyuubi/jdbc/KyuubiDriver.scala
rename to kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/KyuubiDriver.java
index 2e6ff1a..d4e8370 100644
--- a/kyuubi-hive-jdbc/src/main/scala/org/apache/kyuubi/jdbc/KyuubiDriver.scala
+++ b/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/KyuubiDriver.java
@@ -15,25 +15,26 @@
  * limitations under the License.
  */
 
-package org.apache.kyuubi.jdbc
+package org.apache.kyuubi.jdbc;
 
-import java.sql.{Connection, DriverManager, SQLException}
-import java.util.Properties
+import org.apache.hive.jdbc.HiveDriver;
 
-import org.apache.hive.jdbc.HiveDriver
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Properties;
 
-class KyuubiDriver extends HiveDriver {
-  override def connect(url: String, info: Properties): Connection = {
-    if (acceptsURL(url)) {
-      new KyuubiConnection(url, info)
-    } else null
-  }
-}
+public class KyuubiDriver extends HiveDriver {
+    static {
+        try {
+            DriverManager.registerDriver(new KyuubiDriver());
+        } catch (SQLException e) {
+            throw new RuntimeException("Failed to register driver", e);
+        }
+    }
 
-object KyuubiDriver {
-  try {
-    DriverManager.registerDriver(new KyuubiDriver)
-  } catch {
-    case e: SQLException => throw new RuntimeException("Failed to register driver", e)
-  }
+    @Override
+    public Connection connect(String url, Properties info) throws SQLException {
+        return acceptsURL(url) ? new KyuubiConnection(url, info) : null;
+    }
 }
diff --git a/kyuubi-hive-jdbc/src/main/scala/org/apache/kyuubi/jdbc/KyuubiConnection.scala b/kyuubi-hive-jdbc/src/main/scala/org/apache/kyuubi/jdbc/KyuubiConnection.scala
deleted file mode 100644
index 2646815..0000000
--- a/kyuubi-hive-jdbc/src/main/scala/org/apache/kyuubi/jdbc/KyuubiConnection.scala
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.kyuubi.jdbc
-
-import java.sql.{DatabaseMetaData, SQLException}
-import java.util.Properties
-
-import org.apache.hive.jdbc.HiveConnection
-import org.apache.hive.service.rpc.thrift.{TCLIService, TSessionHandle}
-
-class KyuubiConnection(url: String, info: Properties) extends HiveConnection(url, info) {
-
-  override def getMetaData: DatabaseMetaData = {
-    if (isClosed) {
-      throw new SQLException("Connection is closed")
-    } else {
-      val clientField = classOf[HiveConnection].getDeclaredField("client")
-      clientField.setAccessible(true)
-      val client = clientField.get(this).asInstanceOf[TCLIService.Iface]
-      val handleField = classOf[HiveConnection].getDeclaredField("sessHandle")
-      handleField.setAccessible(true)
-      val sessionHandle = handleField.get(this).asInstanceOf[TSessionHandle]
-      new KyuubiDatabaseMetaData(this, client, sessionHandle)
-    }
-  }
-}
diff --git a/kyuubi-hive-jdbc/src/main/scala/org/apache/kyuubi/jdbc/KyuubiDatabaseMetaData.scala b/kyuubi-hive-jdbc/src/main/scala/org/apache/kyuubi/jdbc/KyuubiDatabaseMetaData.scala
deleted file mode 100644
index 77691b8..0000000
--- a/kyuubi-hive-jdbc/src/main/scala/org/apache/kyuubi/jdbc/KyuubiDatabaseMetaData.scala
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.kyuubi.jdbc
-
-import java.sql.{ResultSet, SQLException}
-
-import scala.collection.JavaConverters._
-
-import org.apache.hive.jdbc.{HiveDatabaseMetaData, HiveQueryResultSet}
-import org.apache.hive.service.cli.HiveSQLException
-import org.apache.hive.service.rpc.thrift.{TGetTablesReq, TSessionHandle, TStatusCode}
-import org.apache.hive.service.rpc.thrift.TCLIService.Iface
-import org.apache.thrift.TException
-
-class KyuubiDatabaseMetaData(conn: KyuubiConnection, client: Iface, sessHandle: TSessionHandle)
-  extends HiveDatabaseMetaData(conn, client, sessHandle) {
-  override def getTables(
-      catalog: String,
-      schemaPattern: String,
-      tableNamePattern: String,
-      types: Array[String]): ResultSet = {
-    val getTableReq: TGetTablesReq = new TGetTablesReq(sessHandle)
-    getTableReq.setCatalogName(catalog)
-    getTableReq.setSchemaName(if (schemaPattern == null) "%" else schemaPattern)
-    getTableReq.setTableName(tableNamePattern)
-    if (types != null) {
-      getTableReq.setTableTypes(types.toList.asJava)
-    }
-    val getTableResp = try {
-      client.GetTables(getTableReq)
-    } catch {
-      case e: TException => throw new SQLException(e.getMessage, "08S01", e)
-    }
-    val tStatus = getTableResp.getStatus
-    if (tStatus.getStatusCode != TStatusCode.SUCCESS_STATUS) {
-      throw new HiveSQLException(tStatus)
-    }
-    new HiveQueryResultSet.Builder(conn)
-      .setClient(client)
-      .setSessionHandle(sessHandle)
-      .setStmtHandle(getTableResp.getOperationHandle)
-      .build
-  }
-}
diff --git a/pom.xml b/pom.xml
index ab2755e..5aa03fa 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1084,11 +1084,6 @@
 
     <dependencies>
         <dependency>
-            <groupId>org.scala-lang</groupId>
-            <artifactId>scala-library</artifactId>
-        </dependency>
-
-        <dependency>
             <groupId>org.scalatest</groupId>
             <artifactId>scalatest_${scala.binary.version}</artifactId>
             <scope>test</scope>