wuchong commented on a change in pull request #12290:
URL: https://github.com/apache/flink/pull/12290#discussion_r429759431



##########
File path: 
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/UnsignedTypeConversionITCase.java
##########
@@ -0,0 +1,214 @@
+/*
+ * 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.flink.connector.jdbc.table;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.planner.runtime.utils.TableEnvUtil;
+import org.apache.flink.types.Row;
+
+import ch.vorburger.mariadb4j.DBConfigurationBuilder;
+import ch.vorburger.mariadb4j.junit.MariaDB4jRule;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.flink.table.api.Expressions.row;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test unsigned type conversion between Flink and JDBC driver mysql, the test 
underlying use
+ * MariaDB to mock a DB which use mysql driver too.
+ */
+public class UnsignedTypeConversionITCase {
+
+       private static final String DEFAULT_DB_NAME = "test";
+       private static final String TABLE_NAME = "unsigned_test";
+       private static final String[] DATA = new String[]{
+               "127",
+               "255",
+               "32767",
+               "65535",
+               "2147483647",
+               "4294967295",
+               "9223372036854775807",
+               "18446744073709551615"};
+
+       private StreamTableEnvironment tEnv;
+       private String dbUrl;
+       private Connection connection;
+
+       @ClassRule
+       public static MariaDB4jRule db4jRule = new MariaDB4jRule(
+               DBConfigurationBuilder.newBuilder().setPort(3306).build(),

Review comment:
       Can we use a random port here?

##########
File path: 
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/UnsignedTypeConversionITCase.java
##########
@@ -0,0 +1,214 @@
+/*
+ * 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.flink.connector.jdbc.table;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.planner.runtime.utils.TableEnvUtil;
+import org.apache.flink.types.Row;
+
+import ch.vorburger.mariadb4j.DBConfigurationBuilder;
+import ch.vorburger.mariadb4j.junit.MariaDB4jRule;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.flink.table.api.Expressions.row;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test unsigned type conversion between Flink and JDBC driver mysql, the test 
underlying use
+ * MariaDB to mock a DB which use mysql driver too.
+ */
+public class UnsignedTypeConversionITCase {
+
+       private static final String DEFAULT_DB_NAME = "test";
+       private static final String TABLE_NAME = "unsigned_test";
+       private static final String[] DATA = new String[]{
+               "127",
+               "255",
+               "32767",
+               "65535",
+               "2147483647",
+               "4294967295",
+               "9223372036854775807",
+               "18446744073709551615"};
+
+       private StreamTableEnvironment tEnv;
+       private String dbUrl;
+       private Connection connection;
+
+       @ClassRule
+       public static MariaDB4jRule db4jRule = new MariaDB4jRule(
+               DBConfigurationBuilder.newBuilder().setPort(3306).build(),
+               DEFAULT_DB_NAME,
+               null);
+
+       @Before
+       public void setUp() throws SQLException, ClassNotFoundException {
+               StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+               tEnv = StreamTableEnvironment.create(env);
+               //dbUrl: jdbc:mysql://localhost:3306/test
+               dbUrl = db4jRule.getURL();
+               connection = DriverManager.getConnection(dbUrl);
+               // create mysql table
+               PreparedStatement ddlStatement = 
connection.prepareStatement("create table " + TABLE_NAME + " (" +
+                       " tiny_c TINYINT," +
+                       " tiny_un_c TINYINT UNSIGNED," +
+                       " small_c SMALLINT," +
+                       " small_un_c SMALLINT UNSIGNED," +
+                       " int_c INTEGER ," +
+                       " int_un_c INTEGER UNSIGNED," +
+                       " big_c BIGINT," +
+                       " big_un_c BIGINT UNSIGNED);");
+               ddlStatement.execute();
+
+               // create flink table
+               String sourceDDL = "create table jdbc_table (" +
+                       "tiny_c TINYINT," +
+                       "tiny_un_c SMALLINT," +
+                       "small_c SMALLINT," +
+                       "small_un_c INT," +
+                       "int_c INT," +
+                       "int_un_c BIGINT," +
+                       "big_c BIGINT," +
+                       "big_un_c DECIMAL(20, 0)) with(" +
+                       " 'connector' = 'jdbc'," +
+                       " 'url' = '" + dbUrl + "'," +
+                       " 'table-name' = '" + TABLE_NAME + "'" +
+                       ")";
+               tEnv.executeSql(sourceDDL);
+       }
+
+       @Test
+       public void testReadUnsignedType() throws SQLException {
+               // insert data to db
+               String insertSql = String.format("insert into " + TABLE_NAME +
+                       "(tiny_c, tiny_un_c, small_c, small_un_c, int_c, 
int_un_c, big_c, big_un_c) values (" +
+                       "%s, %s, %s, %s, %s, %s, %s, %s)", DATA);
+               PreparedStatement insertStatement = 
connection.prepareStatement(insertSql);
+               insertStatement.execute();
+               // read data from db
+               Iterator<Row> res = tEnv.executeSql("select tiny_c, tiny_un_c, 
small_c, small_un_c," +
+                       " int_c, int_un_c, big_c, big_un_c from jdbc_table")
+                       .collect();
+               List<Row> actual = new ArrayList<>();
+               while (res.hasNext()) {
+                       actual.add(res.next());
+               }
+               assertTrue(actual.size() == 1);
+               assertTrue(actual.get(0).getArity() == DATA.length);
+               String[] expected = new String[DATA.length];
+               for (int i = 0; i < DATA.length; i++) {
+                       expected[i] = actual.get(0).getField(i).toString();
+               }
+               assertArrayEquals(DATA, expected);
+       }
+
+       @Test
+       public void testWriteUnsignedType() throws Exception {

Review comment:
       Could you combine these two tests into one? We can test writing values 
into the table, and verify the data using jdbc connection. And then test 
reading values from the table. 

##########
File path: 
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/AbstractJdbcRowConverter.java
##########
@@ -114,49 +119,54 @@ protected JdbcDeserializationConverter 
createNullableInternalConverter(LogicalTy
        }
 
        protected JdbcDeserializationConverter 
wrapIntoNullableInternalConverter(JdbcDeserializationConverter 
jdbcDeserializationConverter) {
-               return v -> {
-                       if (v == null) {
+               return (val, sqlType, unsigned) -> {
+                       if (val == null) {
                                return null;
                        } else {
-                               return 
jdbcDeserializationConverter.deserialize(v);
+                               return 
jdbcDeserializationConverter.deserialize(val, sqlType, unsigned);
                        }
                };
        }
 
        protected JdbcDeserializationConverter 
createInternalConverter(LogicalType type) {
                switch (type.getTypeRoot()) {
                        case NULL:
-                               return v -> null;
+                               return (val, sqlType, unsigned) -> null;
                        case BOOLEAN:
-                       case TINYINT:
                        case FLOAT:
                        case DOUBLE:
-                       case INTEGER:
                        case INTERVAL_YEAR_MONTH:
-                       case BIGINT:
                        case INTERVAL_DAY_TIME:
-                               return v -> v;
+                               return (val, sqlType, unsigned) -> val;
+                       case TINYINT:
+                               return (val, sqlType, unsigned) -> ((Integer) 
val).byteValue();
                        case SMALLINT:
                                // Converter for small type that casts value to 
int and then return short value, since
                                // JDBC 1.0 use int type for small values.
-                               return v -> 
(Integer.valueOf(v.toString())).shortValue();
+                               return (val, sqlType, unsigned) -> 
(Integer.valueOf(val.toString())).shortValue();
+                       case INTEGER:
+                               return (val, sqlType, unsigned) -> sqlType == 
Types.SMALLINT && unsigned ? Integer.valueOf(val.toString()) : val;
+                       case BIGINT:
+                               return (val, sqlType, unsigned) -> sqlType == 
Types.INTEGER && unsigned ? Long.valueOf(val.toString()) : val;
+                       case DECIMAL:
+                               final int precision = ((DecimalType) 
type).getPrecision();
+                               final int scale = ((DecimalType) 
type).getScale();
+                               // using decimal(20, 0) to map bigint unsigned
+                               return (val, sqlType, unsigned) -> sqlType == 
Types.BIGINT && unsigned ?
+                                       DecimalData.fromBigDecimal(new 
BigDecimal((BigInteger) val, 0), 20, 0) : 
DecimalData.fromBigDecimal((BigDecimal) val, precision, scale);

Review comment:
       I suggest to use the precision and scale from the `DecimalType` instead 
of hard code `20, 0`, otherwise, if users use a `30, 0` in DDL, the value will 
be corrupt. 

##########
File path: 
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/UnsignedTypeConversionITCase.java
##########
@@ -0,0 +1,214 @@
+/*
+ * 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.flink.connector.jdbc.table;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.planner.runtime.utils.TableEnvUtil;
+import org.apache.flink.types.Row;
+
+import ch.vorburger.mariadb4j.DBConfigurationBuilder;
+import ch.vorburger.mariadb4j.junit.MariaDB4jRule;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.flink.table.api.Expressions.row;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test unsigned type conversion between Flink and JDBC driver mysql, the test 
underlying use
+ * MariaDB to mock a DB which use mysql driver too.
+ */
+public class UnsignedTypeConversionITCase {
+
+       private static final String DEFAULT_DB_NAME = "test";
+       private static final String TABLE_NAME = "unsigned_test";
+       private static final String[] DATA = new String[]{
+               "127",
+               "255",
+               "32767",
+               "65535",
+               "2147483647",
+               "4294967295",
+               "9223372036854775807",
+               "18446744073709551615"};
+
+       private StreamTableEnvironment tEnv;
+       private String dbUrl;
+       private Connection connection;
+
+       @ClassRule
+       public static MariaDB4jRule db4jRule = new MariaDB4jRule(
+               DBConfigurationBuilder.newBuilder().setPort(3306).build(),
+               DEFAULT_DB_NAME,
+               null);
+
+       @Before
+       public void setUp() throws SQLException, ClassNotFoundException {
+               StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+               tEnv = StreamTableEnvironment.create(env);
+               //dbUrl: jdbc:mysql://localhost:3306/test
+               dbUrl = db4jRule.getURL();
+               connection = DriverManager.getConnection(dbUrl);
+               // create mysql table
+               PreparedStatement ddlStatement = 
connection.prepareStatement("create table " + TABLE_NAME + " (" +
+                       " tiny_c TINYINT," +
+                       " tiny_un_c TINYINT UNSIGNED," +
+                       " small_c SMALLINT," +
+                       " small_un_c SMALLINT UNSIGNED," +
+                       " int_c INTEGER ," +
+                       " int_un_c INTEGER UNSIGNED," +
+                       " big_c BIGINT," +
+                       " big_un_c BIGINT UNSIGNED);");
+               ddlStatement.execute();
+
+               // create flink table
+               String sourceDDL = "create table jdbc_table (" +
+                       "tiny_c TINYINT," +
+                       "tiny_un_c SMALLINT," +
+                       "small_c SMALLINT," +
+                       "small_un_c INT," +
+                       "int_c INT," +
+                       "int_un_c BIGINT," +
+                       "big_c BIGINT," +
+                       "big_un_c DECIMAL(20, 0)) with(" +
+                       " 'connector' = 'jdbc'," +
+                       " 'url' = '" + dbUrl + "'," +
+                       " 'table-name' = '" + TABLE_NAME + "'" +
+                       ")";
+               tEnv.executeSql(sourceDDL);
+       }
+
+       @Test
+       public void testReadUnsignedType() throws SQLException {
+               // insert data to db
+               String insertSql = String.format("insert into " + TABLE_NAME +
+                       "(tiny_c, tiny_un_c, small_c, small_un_c, int_c, 
int_un_c, big_c, big_un_c) values (" +
+                       "%s, %s, %s, %s, %s, %s, %s, %s)", DATA);
+               PreparedStatement insertStatement = 
connection.prepareStatement(insertSql);
+               insertStatement.execute();
+               // read data from db
+               Iterator<Row> res = tEnv.executeSql("select tiny_c, tiny_un_c, 
small_c, small_un_c," +
+                       " int_c, int_un_c, big_c, big_un_c from jdbc_table")
+                       .collect();
+               List<Row> actual = new ArrayList<>();
+               while (res.hasNext()) {
+                       actual.add(res.next());
+               }
+               assertTrue(actual.size() == 1);
+               assertTrue(actual.get(0).getArity() == DATA.length);
+               String[] expected = new String[DATA.length];
+               for (int i = 0; i < DATA.length; i++) {
+                       expected[i] = actual.get(0).getField(i).toString();
+               }
+               assertArrayEquals(DATA, expected);

Review comment:
       Can be simplify to ?
   
   ```java
   String[] result = Lists.newArrayList(collected).stream()
                        .map(Row::toString)
                        .toArray(String[]::new);
   assertArrayEquals(DATA, result);
   ```

##########
File path: 
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/UnsignedTypeConversionITCase.java
##########
@@ -0,0 +1,214 @@
+/*
+ * 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.flink.connector.jdbc.table;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.planner.runtime.utils.TableEnvUtil;
+import org.apache.flink.types.Row;
+
+import ch.vorburger.mariadb4j.DBConfigurationBuilder;
+import ch.vorburger.mariadb4j.junit.MariaDB4jRule;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.flink.table.api.Expressions.row;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test unsigned type conversion between Flink and JDBC driver mysql, the test 
underlying use
+ * MariaDB to mock a DB which use mysql driver too.
+ */
+public class UnsignedTypeConversionITCase {

Review comment:
       Please extends `AbstractTestBase` if it is an integration test. 

##########
File path: 
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/AbstractJdbcRowConverter.java
##########
@@ -114,49 +119,54 @@ protected JdbcDeserializationConverter 
createNullableInternalConverter(LogicalTy
        }
 
        protected JdbcDeserializationConverter 
wrapIntoNullableInternalConverter(JdbcDeserializationConverter 
jdbcDeserializationConverter) {
-               return v -> {
-                       if (v == null) {
+               return (val, sqlType, unsigned) -> {
+                       if (val == null) {
                                return null;
                        } else {
-                               return 
jdbcDeserializationConverter.deserialize(v);
+                               return 
jdbcDeserializationConverter.deserialize(val, sqlType, unsigned);
                        }
                };
        }
 
        protected JdbcDeserializationConverter 
createInternalConverter(LogicalType type) {
                switch (type.getTypeRoot()) {
                        case NULL:
-                               return v -> null;
+                               return (val, sqlType, unsigned) -> null;
                        case BOOLEAN:
-                       case TINYINT:
                        case FLOAT:
                        case DOUBLE:
-                       case INTEGER:
                        case INTERVAL_YEAR_MONTH:
-                       case BIGINT:
                        case INTERVAL_DAY_TIME:
-                               return v -> v;
+                               return (val, sqlType, unsigned) -> val;
+                       case TINYINT:
+                               return (val, sqlType, unsigned) -> ((Integer) 
val).byteValue();
                        case SMALLINT:
                                // Converter for small type that casts value to 
int and then return short value, since
                                // JDBC 1.0 use int type for small values.
-                               return v -> 
(Integer.valueOf(v.toString())).shortValue();
+                               return (val, sqlType, unsigned) -> 
(Integer.valueOf(val.toString())).shortValue();

Review comment:
       Use `if (val instanceOf Integer)` instead of string parse. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to