You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2022/12/24 09:10:02 UTC

[doris] 10/15: [fix](jdbc catalog) fix bugs of jdbc catalog and table valued function (#15216)

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

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit ddee3b4b855f5910a3797994050a97ff952339c8
Author: Tiewei Fang <43...@users.noreply.github.com>
AuthorDate: Fri Dec 23 16:46:39 2022 +0800

    [fix](jdbc catalog) fix bugs of jdbc catalog and table valued function (#15216)
    
    * fix bugs
    
    * add `desc function` test
    
    * add test
    
    * fix
---
 .../docker-compose/mysql/init/03-create-table.sql  |  9 ++++++++
 .../docker-compose/mysql/init/04-insert.sql        |  5 +++++
 .../org/apache/doris/analysis/DescribeStmt.java    |  3 ++-
 .../org/apache/doris/external/jdbc/JdbcClient.java | 25 +++++++++++++++++++++-
 .../java/org/apache/doris/qe/ConnectProcessor.java |  2 ++
 .../ExternalFileTableValuedFunction.java           |  2 +-
 .../doris/tablefunction/S3TableValuedFunction.java |  2 +-
 .../table_valued_function/test_hdfs_tvf.out        |  9 ++++++++
 .../jdbc_catalog_p0/test_mysql_jdbc_catalog.out    |  4 ++++
 .../table_valued_function/test_hdfs_tvf.groovy     |  9 ++++++++
 .../jdbc_catalog_p0/test_mysql_jdbc_catalog.groovy |  7 +++++-
 11 files changed, 72 insertions(+), 5 deletions(-)

diff --git a/docker/thirdparties/docker-compose/mysql/init/03-create-table.sql b/docker/thirdparties/docker-compose/mysql/init/03-create-table.sql
index b0e4890013..6c8371e7c7 100644
--- a/docker/thirdparties/docker-compose/mysql/init/03-create-table.sql
+++ b/docker/thirdparties/docker-compose/mysql/init/03-create-table.sql
@@ -214,4 +214,13 @@ create table ex_tb19 (
     timestamp_value timestamp
 ) engine=innodb charset=utf8;
 
+create table doris_test.ex_tb20 (
+    decimal_normal decimal(38, 5),
+    decimal_unsigned decimal(37, 5) unsigned,
+    decimal_out1 decimal(39, 5),
+    decimal_unsigned_out1 decimal(38, 5) unsigned,
+    decimal_long decimal(65, 5),
+    decimal_unsigned_long decimal(65, 5) unsigned
+) engine=innodb charset=utf8;
+
 
diff --git a/docker/thirdparties/docker-compose/mysql/init/04-insert.sql b/docker/thirdparties/docker-compose/mysql/init/04-insert.sql
index 684a4137e2..d978a2298d 100644
--- a/docker/thirdparties/docker-compose/mysql/init/04-insert.sql
+++ b/docker/thirdparties/docker-compose/mysql/init/04-insert.sql
@@ -1119,3 +1119,8 @@ INSERT INTO ex_tb18 VALUES
 
 INSERT INTO ex_tb19 VALUES
 ('2022-11-27', '07:09:51', '2022', '2022-11-27 07:09:51', '2022-11-27 07:09:51');
+
+INSERT INTO doris_test.ex_tb20 VALUES
+(1.12345, 1.12345, 1.12345, 1.12345, 1.12345, 1.12345),
+(123456789012345678901234567890123.12345, 12345678901234567890123456789012.12345, 1234567890123456789012345678901234.12345, 123456789012345678901234567890123.12345,
+123456789012345678901234567890123456789012345678901234567890.12345, 123456789012345678901234567890123456789012345678901234567890.12345);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DescribeStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DescribeStmt.java
index 37370394b9..8a71f8b4be 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DescribeStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DescribeStmt.java
@@ -110,7 +110,8 @@ public class DescribeStmt extends ShowStmt {
     @Override
     public void analyze(Analyzer analyzer) throws UserException {
         if (!isAllTables && isTableValuedFunction) {
-            List<Column> columns = tableValuedFunctionRef.getTableFunction().getTable().getBaseSchema();
+            tableValuedFunctionRef.analyze(analyzer);
+            List<Column> columns = tableValuedFunctionRef.getTable().getBaseSchema();
             for (Column column : columns) {
                 List<String> row = Arrays.asList(
                         column.getDisplayName(),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java b/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
index 9148629d46..0c3b7702eb 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
@@ -21,6 +21,7 @@ import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.Type;
+import org.apache.doris.common.Config;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.util.Util;
 
@@ -78,6 +79,7 @@ public class JdbcClient {
         this.dbType = parseDbType(jdbcUrl);
         this.checkSum = computeObjectChecksum();
 
+        ClassLoader oldClassLoader = Thread.currentThread().getContextClassLoader();
         try {
             // TODO(ftw): The problem here is that the jar package is handled by FE
             //  and URLClassLoader may load the jar package directly into memory
@@ -94,6 +96,8 @@ public class JdbcClient {
             dataSource = new HikariDataSource(config);
         } catch (MalformedURLException e) {
             throw new JdbcClientException("MalformedURLException to load class about " + driverUrl, e);
+        } finally {
+            Thread.currentThread().setContextClassLoader(oldClassLoader);
         }
     }
 
@@ -338,6 +342,17 @@ public class JdbcClient {
                     return Type.BIGINT;
                 case "BIGINT":
                     return ScalarType.createStringType();
+                case "DECIMAL":
+                    int precision = fieldSchema.getColumnSize() + 1;
+                    int scale = fieldSchema.getDecimalDigits();
+                    if (precision <= ScalarType.MAX_DECIMAL128_PRECISION) {
+                        if (!Config.enable_decimal_conversion && precision > ScalarType.MAX_DECIMALV2_PRECISION) {
+                            return ScalarType.createStringType();
+                        }
+                        return ScalarType.createDecimalType(precision, scale);
+                    } else {
+                        return ScalarType.createStringType();
+                    }
                 default:
                     throw new JdbcClientException("Unknown UNSIGNED type of mysql, type: [" + mysqlType + "]");
             }
@@ -368,7 +383,14 @@ public class JdbcClient {
             case "DECIMAL":
                 int precision = fieldSchema.getColumnSize();
                 int scale = fieldSchema.getDecimalDigits();
-                return ScalarType.createDecimalType(precision, scale);
+                if (precision <= ScalarType.MAX_DECIMAL128_PRECISION) {
+                    if (!Config.enable_decimal_conversion && precision > ScalarType.MAX_DECIMALV2_PRECISION) {
+                        return ScalarType.createStringType();
+                    }
+                    return ScalarType.createDecimalType(precision, scale);
+                } else {
+                    return ScalarType.createStringType();
+                }
             case "CHAR":
                 ScalarType charType = ScalarType.createType(PrimitiveType.CHAR);
                 charType.setLength(fieldSchema.columnSize);
@@ -392,6 +414,7 @@ public class JdbcClient {
             case "BIT":
             case "BINARY":
             case "VARBINARY":
+            case "ENUM":
                 return ScalarType.createStringType();
             default:
                 throw new JdbcClientException("Can not convert mysql data type to doris data type for type ["
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java
index 2715aa3a6c..54de1449d0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java
@@ -429,6 +429,8 @@ public class ConnectProcessor {
                 channel.sendOnePacket(serializer.toByteBuffer());
             }
 
+        } catch (Throwable throwable) {
+            handleQueryException(throwable, "", null, null);
         } finally {
             table.readUnlock();
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java
index 4abed5362f..7cadc26db2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/ExternalFileTableValuedFunction.java
@@ -141,7 +141,7 @@ public abstract class ExternalFileTableValuedFunction extends TableValuedFunctio
         try {
             BrokerUtil.parseFile(path, brokerDesc, fileStatuses);
         } catch (UserException e) {
-            throw new AnalysisException("parse file failed, path = " + path);
+            throw new AnalysisException("parse file failed, path = " + path, e);
         }
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java
index e0b4e310f5..ccf5c3eac5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/S3TableValuedFunction.java
@@ -85,7 +85,7 @@ public class S3TableValuedFunction extends ExternalFileTableValuedFunction {
         try {
             s3uri = S3URI.create(validParams.get(S3_URI), forceVirtualHosted);
         } catch (UserException e) {
-            throw new AnalysisException("parse s3 uri failed, uri = " + validParams.get(S3_URI));
+            throw new AnalysisException("parse s3 uri failed, uri = " + validParams.get(S3_URI), e);
         }
         if (forceVirtualHosted) {
             // s3uri.getVirtualBucket() is: virtualBucket.endpoint, Eg:
diff --git a/regression-test/data/correctness_p0/table_valued_function/test_hdfs_tvf.out b/regression-test/data/correctness_p0/table_valued_function/test_hdfs_tvf.out
index 6ed17594b9..e18f42d8ff 100644
--- a/regression-test/data/correctness_p0/table_valued_function/test_hdfs_tvf.out
+++ b/regression-test/data/correctness_p0/table_valued_function/test_hdfs_tvf.out
@@ -287,3 +287,12 @@
 4	shenzhen	2345674
 5	guangzhou	2345675
 
+-- !desc --
+s_suppkey	INT	Yes	false	\N	NONE
+s_name	TEXT	Yes	false	\N	NONE
+s_address	TEXT	Yes	false	\N	NONE
+s_nationkey	INT	Yes	false	\N	NONE
+s_phone	TEXT	Yes	false	\N	NONE
+s_acctbal	DECIMAL(9,0)	Yes	false	\N	NONE
+s_comment	TEXT	Yes	false	\N	NONE
+
diff --git a/regression-test/data/jdbc_catalog_p0/test_mysql_jdbc_catalog.out b/regression-test/data/jdbc_catalog_p0/test_mysql_jdbc_catalog.out
index 9c4a43e959..00360cb185 100644
--- a/regression-test/data/jdbc_catalog_p0/test_mysql_jdbc_catalog.out
+++ b/regression-test/data/jdbc_catalog_p0/test_mysql_jdbc_catalog.out
@@ -152,3 +152,7 @@ bca	2022-11-02	2022-11-02	8012	vivo
 -- !ex_tb19 --
 2022-11-27	07:09:51	2022	2022-11-27T07:09:51	2022-11-27T07:09:51
 
+-- !ex_tb20 --
+1.12345	1.12345	1.12345	1.12345	1.12345	1.12345
+123456789012345678901234567890123.12345	12345678901234567890123456789012.12345	1234567890123456789012345678901234.12345	123456789012345678901234567890123.12345	123456789012345678901234567890123456789012345678901234567890.12345	123456789012345678901234567890123456789012345678901234567890.12345
+
diff --git a/regression-test/suites/correctness_p0/table_valued_function/test_hdfs_tvf.groovy b/regression-test/suites/correctness_p0/table_valued_function/test_hdfs_tvf.groovy
index 07b8363b0b..4c4f940026 100644
--- a/regression-test/suites/correctness_p0/table_valued_function/test_hdfs_tvf.groovy
+++ b/regression-test/suites/correctness_p0/table_valued_function/test_hdfs_tvf.groovy
@@ -191,6 +191,15 @@ suite("test_hdfs_tvf") {
             sql "sync"
             assertTrue(result2[0][0] == 5, "Insert should update 12 rows")
             qt_insert """ select * from test_hdfs_tvf order by id; """
+
+            // test desc function
+            uri = "${defaultFS}" + "/user/doris/preinstalled_data/hdfs_tvf/test_parquet.snappy.parquet"
+            format = "parquet"
+            qt_desc """ desc function HDFS(
+                            "uri" = "${uri}",
+                            "fs.defaultFS"= "${defaultFS}",
+                            "hadoop.username" = "${hdfsUserName}",
+                            "format" = "${format}"); """
         } finally {
         }
     }
diff --git a/regression-test/suites/jdbc_catalog_p0/test_mysql_jdbc_catalog.groovy b/regression-test/suites/jdbc_catalog_p0/test_mysql_jdbc_catalog.groovy
index 97922eee57..77f7108275 100644
--- a/regression-test/suites/jdbc_catalog_p0/test_mysql_jdbc_catalog.groovy
+++ b/regression-test/suites/jdbc_catalog_p0/test_mysql_jdbc_catalog.groovy
@@ -43,8 +43,9 @@ suite("test_mysql_jdbc_catalog", "p0") {
         String ex_tb17 = "ex_tb17";
         String ex_tb18 = "ex_tb18";
         String ex_tb19 = "ex_tb19";
+        String ex_tb20 = "ex_tb20";
 
-
+        sql """ADMIN SET FRONTEND CONFIG ("enable_decimal_conversion" = "true");"""
         sql """drop catalog if exists ${catalog_name} """
 
         // if use 'com.mysql.cj.jdbc.Driver' here, it will report: ClassNotFound
@@ -94,5 +95,9 @@ suite("test_mysql_jdbc_catalog", "p0") {
         order_qt_ex_tb17  """ select * from ${ex_tb17} order by id; """
         order_qt_ex_tb18  """ select * from ${ex_tb18} order by num_tinyint; """
         order_qt_ex_tb19  """ select * from ${ex_tb19} order by date_value; """
+        order_qt_ex_tb20  """ select * from ${ex_tb20} order by decimal_normal; """
+
+        sql """drop catalog if exists ${catalog_name} """
+        sql """drop resource if exists jdbc_resource_catalog_mysql"""
     }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org