You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/01/02 09:24:51 UTC

[GitHub] [flink] docete opened a new pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

docete opened a new pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745
 
 
   …s with precision (or/and scale)
   
   ## What is the purpose of the change
   
   JDBC table source didn't work for types with precision (and/or scale) in blink planner, such as TIMESTAMP(6), DECIMAL(10, 4). The ValidationException described in FLINK-15445 would be thrown. The root cause is JDBCTableSource didn't override `getProducedDataType` interface, and returns wrong datatype. This PR fix it and add an itcase to verify.
   
   ## Brief change log
   
   - 6daa934 Override `getProducedDataType` for JDBCTableSource and add tests to verify
   
   ## Verifying this change
   
   This change added tests 
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (**yes** / no)
     - If yes, how is the feature documented? (not applicable / docs / **JavaDocs** / not documented)
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376195245
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java
 ##########
 @@ -73,17 +75,23 @@ private JDBCTableSource(
 		this.selectFields = selectFields;
 
 		final TypeInformation<?>[] schemaTypeInfos = schema.getFieldTypes();
+		final DataType[] schemaDataTypes = schema.getFieldDataTypes();
 		final String[] schemaFieldNames = schema.getFieldNames();
 		if (selectFields != null) {
 			TypeInformation<?>[] typeInfos = new TypeInformation[selectFields.length];
-			String[] typeNames = new String[selectFields.length];
+			DataType[] dataTypes = new DataType[selectFields.length];
+			String[] fieldNames = new String[selectFields.length];
 			for (int i = 0; i < selectFields.length; i++) {
 				typeInfos[i] = schemaTypeInfos[selectFields[i]];
-				typeNames[i] = schemaFieldNames[selectFields[i]];
+				dataTypes[i] = schemaDataTypes[selectFields[i]];
+				fieldNames[i] = schemaFieldNames[selectFields[i]];
 			}
-			this.returnType = new RowTypeInfo(typeInfos, typeNames);
+			this.returnType = new RowTypeInfo(typeInfos, fieldNames);
 
 Review comment:
   Can we remove the `returnType` member field?
   It's error-prone to maintain two objects. The returnType is only used in `getDataStream` and can be derived via `TypeConversions.fromDataTypeToLegacyInfo(producedDataType)`.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376205505
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceITCase.java
 ##########
 @@ -18,93 +18,167 @@
 
 package org.apache.flink.api.java.io.jdbc;
 
-import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.api.java.StreamTableEnvironment;
 import org.apache.flink.table.runtime.utils.StreamITCase;
+import org.apache.flink.test.util.AbstractTestBase;
 import org.apache.flink.types.Row;
 
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
-import java.util.ArrayList;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
 import java.util.List;
 
+
 /**
- * IT case for {@link JDBCTableSource}.
+ * ITCase for {@link JDBCTableSource}.
  */
-public class JDBCTableSourceITCase extends JDBCTestBase {
-
-	private static final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-	private static final EnvironmentSettings bsSettings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build();
-	private static final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, bsSettings);
-
-	static final String TABLE_SOURCE_SQL = "CREATE TABLE books (" +
-		" id int, " +
-		" title varchar, " +
-		" author varchar, " +
-		" price double, " +
-		" qty int " +
-		") with (" +
-		" 'connector.type' = 'jdbc', " +
-		" 'connector.url' = 'jdbc:derby:memory:ebookshop', " +
-		" 'connector.table' = 'books', " +
-		" 'connector.driver' = 'org.apache.derby.jdbc.EmbeddedDriver' " +
-		")";
-
-	@BeforeClass
-	public static void createTable() {
-		tEnv.sqlUpdate(TABLE_SOURCE_SQL);
+public class JDBCTableSourceITCase extends AbstractTestBase {
+
+	public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver";
+	public static final String DB_URL = "jdbc:derby:memory:test";
+	public static final String INPUT_TABLE = "jdbcSource";
+
+	@Before
+	public void before() throws ClassNotFoundException, SQLException {
+		System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL");
+		Class.forName(DRIVER_CLASS);
+
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL + ";create=true");
+			Statement statement = conn.createStatement()) {
+			statement.executeUpdate("CREATE TABLE " + INPUT_TABLE + " (" +
+					"id BIGINT NOT NULL," +
+					"timestamp6_col TIMESTAMP, " +
+					"timestamp9_col TIMESTAMP, " +
+					"time_col TIME, " +
+					"real_col FLOAT(23), " +    // A precision of 23 or less makes FLOAT equivalent to REAL.
+					"double_col FLOAT(24)," +   // A precision of 24 or greater makes FLOAT equivalent to DOUBLE PRECISION.
+					"decimal_col DECIMAL(10, 4))");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"1, TIMESTAMP('2020-01-01 15:35:00.123456'), TIMESTAMP('2020-01-01 15:35:00.123456789'), " +
+					"TIME('15:35:00'), 1.175E-37, 1.79769E+308, 100.1234)");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"2, TIMESTAMP('2020-01-01 15:36:01.123456'), TIMESTAMP('2020-01-01 15:36:01.123456789'), " +
+					"TIME('15:36:01'), -1.175E-37, -1.79769E+308, 101.1234)");
+		}
+	}
+
+	@After
+	public void clearOutputTable() throws Exception {
+		Class.forName(DRIVER_CLASS);
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL);
+			Statement stat = conn.createStatement()) {
+			stat.execute("DROP TABLE " + INPUT_TABLE);
 
 Review comment:
   Should we use `stat.executeUdpate`?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r377585237
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java
 ##########
 @@ -46,10 +59,84 @@
 		return Optional.empty();
 	}
 
-	private static class DerbyDialect implements JDBCDialect {
+	private abstract static class AbstractDialect implements JDBCDialect {
+
+		@Override
+		public void validate(TableSchema schema) throws ValidationException {
+			for (int i = 0; i < schema.getFieldCount(); i++) {
+				DataType dt = schema.getFieldDataType(i).get();
+				String fieldName = schema.getFieldName(i).get();
+
+				// TODO: We can't convert VARBINARY(n) data type to
+				//  PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO in LegacyTypeInfoDataTypeConverter
+				//  when n is smaller than Integer.MAX_VALUE
+				if (unsupportedTypes().contains(dt.getLogicalType().getTypeRoot()) ||
+						(!(dt.getLogicalType() instanceof LegacyTypeInformationType) &&
+						(VARBINARY == dt.getLogicalType().getTypeRoot()
 
 Review comment:
   Yes, I think so. Will update soon.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r362691932
 
 

 ##########
 File path: flink-connectors/flink-jdbc/pom.xml
 ##########
 @@ -82,5 +82,20 @@ under the License.
 			<type>test-jar</type>
 			<scope>test</scope>
 		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table-planner-blink_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
 
 Review comment:
   https://issues.apache.org/jira/browse/FLINK-15460

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376203705
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java
 ##########
 @@ -70,11 +105,33 @@ public String quoteIdentifier(String identifier) {
 
 		private static final long serialVersionUID = 1L;
 
+		private static final int MAX_MYSQL_TIMESTAMP_PRECISION = 6;
+
+		private static final int MIN_MYSQL_TIMESTAMP_PRECISION = 0;
+
 		@Override
 		public boolean canHandle(String url) {
 			return url.startsWith("jdbc:mysql:");
 		}
 
+		@Override
+		public void validate(TableSchema schema) {
+			for (int i = 0; i < schema.getFieldCount(); i++) {
+				DataType dt = schema.getFieldDataType(i).get();
+				String fieldName = schema.getFieldName(i).get();
+				if (TIMESTAMP_WITHOUT_TIME_ZONE == dt.getLogicalType().getTypeRoot()) {
+					int precision = ((TimestampType) dt.getLogicalType()).getPrecision();
+					if (precision > MAX_MYSQL_TIMESTAMP_PRECISION) {
+						throw new ValidationException(
+								String.format("The precision of %s is out of range [%d, %d].",
+										fieldName,
+										MIN_MYSQL_TIMESTAMP_PRECISION,
+										MAX_MYSQL_TIMESTAMP_PRECISION));
+					}
+				}
 
 Review comment:
   It seems that the validation logic is the same, maybe we can refactor it a bit more to have a `AbstractJDBCDialect` which implements `JDBCDialect`.
   
   ```java
   	private abstract static class AbstractDialect implements JDBCDialect {
   
   		@Override
   		public void validate(TableSchema schema) throws ValidationException {
   			// implement the common validation logic here
   		}
   		
   		public abstract int maxDecimalPrecision();
   		
   		public abstract int minDecimalPrecision();
   		
   		public abstract int maxTimestampPrecision();
   		
   		public abstract int minTimestampPrecision();
   		
   		public abstract List<LogicalTypeRoot> unsupportedTypes();
   	}
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r377440051
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java
 ##########
 @@ -46,10 +59,84 @@
 		return Optional.empty();
 	}
 
-	private static class DerbyDialect implements JDBCDialect {
+	private abstract static class AbstractDialect implements JDBCDialect {
+
+		@Override
+		public void validate(TableSchema schema) throws ValidationException {
+			for (int i = 0; i < schema.getFieldCount(); i++) {
+				DataType dt = schema.getFieldDataType(i).get();
+				String fieldName = schema.getFieldName(i).get();
+
+				// TODO: We can't convert VARBINARY(n) data type to
+				//  PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO in LegacyTypeInfoDataTypeConverter
+				//  when n is smaller than Integer.MAX_VALUE
+				if (unsupportedTypes().contains(dt.getLogicalType().getTypeRoot()) ||
+						(!(dt.getLogicalType() instanceof LegacyTypeInformationType) &&
+						(VARBINARY == dt.getLogicalType().getTypeRoot()
 
 Review comment:
   Could we just simply `dt.getLogicalType() instanceof VarBinaryType ` to match it is a VarBinaryType? I think currently there isn't a `LegacyTypeInformationType` which is VARBINARY.  The same to the below if branches.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r362465196
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java
 ##########
 @@ -112,6 +120,11 @@ public boolean isBounded() {
 		return returnType;
 	}
 
+	@Override
+	public DataType getProducedDataType() {
 
 Review comment:
   Please remove the overrided implementation of `getReturnType()`

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570154489
 
 
   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 6daa934e565422f91003cde39df125268119171a (Thu Jan 02 09:27:11 UTC 2020)
   
   **Warnings:**
    * **1 pom.xml files were touched**: Check for build and licensing issues.
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r374548100
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceITCase.java
 ##########
 @@ -0,0 +1,146 @@
+/*
+ * 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.api.java.io.jdbc;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.java.StreamTableEnvironment;
+import org.apache.flink.table.runtime.utils.StreamITCase;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.List;
+
+
+/**
+ * ITCase for {@link JDBCTableSource}.
+ */
+public class JDBCTableSourceITCase extends AbstractTestBase {
+
+	public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver";
+	public static final String DB_URL = "jdbc:derby:memory:test";
+	public static final String INPUT_TABLE = "jdbcSource";
+
+	@Before
+	public void before() throws ClassNotFoundException, SQLException {
+		System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL");
+		Class.forName(DRIVER_CLASS);
+
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL + ";create=true");
+			Statement statement = conn.createStatement()) {
+			statement.executeUpdate("CREATE TABLE " + INPUT_TABLE + " (" +
+					"id BIGINT NOT NULL," +
+					"timestamp6_col TIMESTAMP, " +
+					"time_col TIME, " +
+					"decimal_col DECIMAL(10, 4))");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"1, TIMESTAMP('2020-01-01 15:35:00.123456'), TIME('15:35:00'), 100.1234)");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"2, TIMESTAMP('2020-01-01 15:36:01.123456'), TIME('15:36:01'), 101.1234)");
+		}
+	}
+
+	@After
+	public void clearOutputTable() throws Exception {
+		Class.forName(DRIVER_CLASS);
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL);
+			Statement stat = conn.createStatement()) {
+			stat.execute("DROP TABLE " + INPUT_TABLE);
+		}
+	}
+
+	@Test
+	public void testJDBCSource() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
+			.useBlinkPlanner()
+			.inStreamingMode()
+			.build();
+		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
+
+		tEnv.sqlUpdate(
+			"CREATE TABLE " + INPUT_TABLE + "(" +
+				"id BIGINT," +
+				"timestamp6_col TIMESTAMP(6)," +
+				"time_col TIME," +
+				"decimal_col DECIMAL(10, 4)" +
 
 Review comment:
   Have add more types for derby. and the combining of source and sink integrate tests will postpone to since the sinks did not support new type system. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-584969563
 
 
   Btw, please remove the unused imports in `JDBCDialects`. The travis is failed. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376199069
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialect.java
 ##########
 @@ -35,6 +37,15 @@
 	 */
 	boolean canHandle(String url);
 
+	/**
+	 * Check if this dialect instance support a specific data type in table schema.
+	 *
+	 * @param schema the table schema
+	 */
+	default void validate(TableSchema schema) {
 
 Review comment:
   throws `ValidationException` on the method signature. And please add a description about the exception in the javadoc.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147345500 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148145921 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/148156951 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   * f7456ac7e14681e18199941e0962b207eb99bef9 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/147345500) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813) 
   * 6336abbade5c71df3bd26fad40209e89642810d8 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148145921) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990) 
   * 330adf7ac3582db6b47a093e6f342351a3ba7e43 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/148156951) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376894169
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceITCase.java
 ##########
 @@ -18,93 +18,167 @@
 
 package org.apache.flink.api.java.io.jdbc;
 
-import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.api.java.StreamTableEnvironment;
 import org.apache.flink.table.runtime.utils.StreamITCase;
+import org.apache.flink.test.util.AbstractTestBase;
 import org.apache.flink.types.Row;
 
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
-import java.util.ArrayList;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
 import java.util.List;
 
+
 /**
- * IT case for {@link JDBCTableSource}.
+ * ITCase for {@link JDBCTableSource}.
  */
-public class JDBCTableSourceITCase extends JDBCTestBase {
-
-	private static final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-	private static final EnvironmentSettings bsSettings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build();
-	private static final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, bsSettings);
-
-	static final String TABLE_SOURCE_SQL = "CREATE TABLE books (" +
-		" id int, " +
-		" title varchar, " +
-		" author varchar, " +
-		" price double, " +
-		" qty int " +
-		") with (" +
-		" 'connector.type' = 'jdbc', " +
-		" 'connector.url' = 'jdbc:derby:memory:ebookshop', " +
-		" 'connector.table' = 'books', " +
-		" 'connector.driver' = 'org.apache.derby.jdbc.EmbeddedDriver' " +
-		")";
-
-	@BeforeClass
-	public static void createTable() {
-		tEnv.sqlUpdate(TABLE_SOURCE_SQL);
+public class JDBCTableSourceITCase extends AbstractTestBase {
+
+	public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver";
+	public static final String DB_URL = "jdbc:derby:memory:test";
+	public static final String INPUT_TABLE = "jdbcSource";
+
+	@Before
+	public void before() throws ClassNotFoundException, SQLException {
+		System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL");
 
 Review comment:
   We add this to get rid of derby.log or the UT will end up with the derby.log file in the root of the project (flink-jdbc).

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r362464519
 
 

 ##########
 File path: flink-connectors/flink-jdbc/pom.xml
 ##########
 @@ -82,5 +82,20 @@ under the License.
 			<type>test-jar</type>
 			<scope>test</scope>
 		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table-planner-blink_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
 
 Review comment:
   We can mark it as provided and put besides old planner dependency.
   
   ```xml
   <scope>provided</scope>
   <optional>true</optional>
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r362693159
 
 

 ##########
 File path: flink-connectors/flink-jdbc/pom.xml
 ##########
 @@ -82,5 +82,20 @@ under the License.
 			<type>test-jar</type>
 			<scope>test</scope>
 		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table-planner-blink_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
 
 Review comment:
   Thanks.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376196133
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceSinkFactory.java
 ##########
 @@ -130,8 +130,11 @@
 		TableSchema schema = TableSchemaUtils.getPhysicalSchema(
 			descriptorProperties.getTableSchema(SCHEMA));
 
+		JDBCOptions options = getJDBCOptions(descriptorProperties);
+		options.getDialect().validate(schema);
 
 Review comment:
   I think it would be better to move the validation logic into `JDBCValidator`.
   https://github.com/apache/flink/blob/master/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/table/descriptors/JDBCValidator.java#L73

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147345500 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148145921 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148156951 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   Hash:5e538c4875e6241d494585c6e2a8f586ac078131 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148370442 TriggerType:PUSH TriggerID:5e538c4875e6241d494585c6e2a8f586ac078131
   Hash:5e538c4875e6241d494585c6e2a8f586ac078131 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5056 TriggerType:PUSH TriggerID:5e538c4875e6241d494585c6e2a8f586ac078131
   Hash:7790797e5e90af02bc94b48f5326077573600f25 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5082 TriggerType:PUSH TriggerID:7790797e5e90af02bc94b48f5326077573600f25
   Hash:7790797e5e90af02bc94b48f5326077573600f25 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/148511660 TriggerType:PUSH TriggerID:7790797e5e90af02bc94b48f5326077573600f25
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   * f7456ac7e14681e18199941e0962b207eb99bef9 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/147345500) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813) 
   * 6336abbade5c71df3bd26fad40209e89642810d8 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148145921) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990) 
   * 330adf7ac3582db6b47a093e6f342351a3ba7e43 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148156951) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992) 
   * 5e538c4875e6241d494585c6e2a8f586ac078131 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148370442) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5056) 
   * 7790797e5e90af02bc94b48f5326077573600f25 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/148511660) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5082) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] JingsongLi commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r362411634
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java
 ##########
 @@ -112,6 +113,11 @@ public boolean isBounded() {
 		return returnType;
 	}
 
+	@Override
+	public DataType getProducedDataType() {
+		return schema.toRowDataType();
 
 Review comment:
   This is wrong fix, you should modify `returnType`.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r362693737
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceITCase.java
 ##########
 @@ -0,0 +1,146 @@
+/*
+ * 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.api.java.io.jdbc;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.java.StreamTableEnvironment;
+import org.apache.flink.table.runtime.utils.StreamITCase;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.List;
+
+
+/**
+ * ITCase for {@link JDBCTableSource}.
+ */
+public class JDBCTableSourceITCase extends AbstractTestBase {
+
+	public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver";
+	public static final String DB_URL = "jdbc:derby:memory:test";
+	public static final String INPUT_TABLE = "jdbcSource";
+
+	@Before
+	public void before() throws ClassNotFoundException, SQLException {
+		System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL");
+		Class.forName(DRIVER_CLASS);
+
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL + ";create=true");
+			Statement statement = conn.createStatement()) {
+			statement.executeUpdate("CREATE TABLE " + INPUT_TABLE + " (" +
+					"id BIGINT NOT NULL," +
+					"timestamp6_col TIMESTAMP, " +
+					"time_col TIME, " +
+					"decimal_col DECIMAL(10, 4))");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"1, TIMESTAMP('2020-01-01 15:35:00.123456'), TIME('15:35:00'), 100.1234)");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"2, TIMESTAMP('2020-01-01 15:36:01.123456'), TIME('15:36:01'), 101.1234)");
+		}
+	}
+
+	@After
+	public void clearOutputTable() throws Exception {
+		Class.forName(DRIVER_CLASS);
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL);
+			Statement stat = conn.createStatement()) {
+			stat.execute("DROP TABLE " + INPUT_TABLE);
+		}
+	}
+
+	@Test
+	public void testJDBCSource() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
+			.useBlinkPlanner()
+			.inStreamingMode()
+			.build();
+		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
+
+		tEnv.sqlUpdate(
+			"CREATE TABLE " + INPUT_TABLE + "(" +
+				"id BIGINT," +
+				"timestamp6_col TIMESTAMP(6)," +
+				"time_col TIME," +
+				"decimal_col DECIMAL(10, 4)" +
 
 Review comment:
   Could you add full list of types to have a full coverage? For example, add `TIMESTAMP`, `TIMESTAMP(9)`, `DECIMAL(38, 18)`, `DECIMAL`, `FLOAT` (we have a bug for float before), etc...
   
   I would also suggest to combine source integrate tests and sink integrate tests, e.g.  read from collections and write into jdbc using SQL, and read from JDBC to verify the result. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376204154
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java
 ##########
 @@ -50,11 +60,36 @@
 
 		private static final long serialVersionUID = 1L;
 
+		private static final int MAX_DERBY_DECIMAL_PRECISION = 31;
+
+		private static final int MIN_DERBY_DECIMAL_PRECISION = 1;
 
 Review comment:
   Could you add a comment above these constants that includes documentation link describes the precision? So that we can have the single truth.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   * f7456ac7e14681e18199941e0962b207eb99bef9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376206438
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceITCase.java
 ##########
 @@ -18,93 +18,167 @@
 
 package org.apache.flink.api.java.io.jdbc;
 
-import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.api.java.StreamTableEnvironment;
 import org.apache.flink.table.runtime.utils.StreamITCase;
+import org.apache.flink.test.util.AbstractTestBase;
 import org.apache.flink.types.Row;
 
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
-import java.util.ArrayList;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
 import java.util.List;
 
+
 /**
- * IT case for {@link JDBCTableSource}.
+ * ITCase for {@link JDBCTableSource}.
  */
-public class JDBCTableSourceITCase extends JDBCTestBase {
-
-	private static final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-	private static final EnvironmentSettings bsSettings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build();
-	private static final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, bsSettings);
-
-	static final String TABLE_SOURCE_SQL = "CREATE TABLE books (" +
-		" id int, " +
-		" title varchar, " +
-		" author varchar, " +
-		" price double, " +
-		" qty int " +
-		") with (" +
-		" 'connector.type' = 'jdbc', " +
-		" 'connector.url' = 'jdbc:derby:memory:ebookshop', " +
-		" 'connector.table' = 'books', " +
-		" 'connector.driver' = 'org.apache.derby.jdbc.EmbeddedDriver' " +
-		")";
-
-	@BeforeClass
-	public static void createTable() {
-		tEnv.sqlUpdate(TABLE_SOURCE_SQL);
+public class JDBCTableSourceITCase extends AbstractTestBase {
+
+	public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver";
+	public static final String DB_URL = "jdbc:derby:memory:test";
+	public static final String INPUT_TABLE = "jdbcSource";
+
+	@Before
+	public void before() throws ClassNotFoundException, SQLException {
+		System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL");
 
 Review comment:
   Do we really need this? Is there any error messages thrown when run these tests?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r362691344
 
 

 ##########
 File path: flink-connectors/flink-jdbc/pom.xml
 ##########
 @@ -82,5 +82,20 @@ under the License.
 			<type>test-jar</type>
 			<scope>test</scope>
 		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table-planner-blink_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
 
 Review comment:
   The connector code should not depend on planner now. Only testing code depend on them. I will open a ticket to remove planner dependencies from connectors by changing the scope to test.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] libenchao commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
libenchao commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376680959
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceITCase.java
 ##########
 @@ -18,93 +18,167 @@
 
 package org.apache.flink.api.java.io.jdbc;
 
-import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.api.java.StreamTableEnvironment;
 import org.apache.flink.table.runtime.utils.StreamITCase;
+import org.apache.flink.test.util.AbstractTestBase;
 import org.apache.flink.types.Row;
 
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
-import java.util.ArrayList;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
 import java.util.List;
 
+
 /**
- * IT case for {@link JDBCTableSource}.
+ * ITCase for {@link JDBCTableSource}.
  */
-public class JDBCTableSourceITCase extends JDBCTestBase {
-
-	private static final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-	private static final EnvironmentSettings bsSettings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build();
-	private static final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, bsSettings);
-
-	static final String TABLE_SOURCE_SQL = "CREATE TABLE books (" +
-		" id int, " +
-		" title varchar, " +
-		" author varchar, " +
-		" price double, " +
-		" qty int " +
-		") with (" +
-		" 'connector.type' = 'jdbc', " +
-		" 'connector.url' = 'jdbc:derby:memory:ebookshop', " +
-		" 'connector.table' = 'books', " +
-		" 'connector.driver' = 'org.apache.derby.jdbc.EmbeddedDriver' " +
-		")";
-
-	@BeforeClass
-	public static void createTable() {
-		tEnv.sqlUpdate(TABLE_SOURCE_SQL);
+public class JDBCTableSourceITCase extends AbstractTestBase {
+
+	public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver";
+	public static final String DB_URL = "jdbc:derby:memory:test";
+	public static final String INPUT_TABLE = "jdbcSource";
+
+	@Before
+	public void before() throws ClassNotFoundException, SQLException {
+		System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL");
+		Class.forName(DRIVER_CLASS);
+
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL + ";create=true");
+			Statement statement = conn.createStatement()) {
+			statement.executeUpdate("CREATE TABLE " + INPUT_TABLE + " (" +
+					"id BIGINT NOT NULL," +
+					"timestamp6_col TIMESTAMP, " +
+					"timestamp9_col TIMESTAMP, " +
+					"time_col TIME, " +
+					"real_col FLOAT(23), " +    // A precision of 23 or less makes FLOAT equivalent to REAL.
+					"double_col FLOAT(24)," +   // A precision of 24 or greater makes FLOAT equivalent to DOUBLE PRECISION.
+					"decimal_col DECIMAL(10, 4))");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"1, TIMESTAMP('2020-01-01 15:35:00.123456'), TIMESTAMP('2020-01-01 15:35:00.123456789'), " +
+					"TIME('15:35:00'), 1.175E-37, 1.79769E+308, 100.1234)");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"2, TIMESTAMP('2020-01-01 15:36:01.123456'), TIMESTAMP('2020-01-01 15:36:01.123456789'), " +
+					"TIME('15:36:01'), -1.175E-37, -1.79769E+308, 101.1234)");
+		}
+	}
+
+	@After
+	public void clearOutputTable() throws Exception {
+		Class.forName(DRIVER_CLASS);
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL);
+			Statement stat = conn.createStatement()) {
+			stat.execute("DROP TABLE " + INPUT_TABLE);
+		}
 	}
 
 	@Test
-	public void testFieldsProjection() throws Exception {
-		StreamITCase.clear();
+	public void testJDBCSource() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
+			.useBlinkPlanner()
+			.inStreamingMode()
+			.build();
+		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
+
+		tEnv.sqlUpdate(
+			"CREATE TABLE " + INPUT_TABLE + "(" +
+				"id BIGINT," +
+				"timestamp6_col TIMESTAMP(6)," +
+				"timestamp9_col TIMESTAMP(9)," +
+				"time_col TIME," +
+				"real_col FLOAT," +
+				"double_col DOUBLE," +
+				"decimal_col DECIMAL(10, 4)" +
+				") WITH (" +
+				"  'connector.type'='jdbc'," +
+				"  'connector.url'='" + DB_URL + "'," +
+				"  'connector.table'='" + INPUT_TABLE + "'" +
+				")"
+		);
 
-		Table result = tEnv.sqlQuery(SELECT_ID_BOOKS);
-		DataStream<Row> resultSet = tEnv.toAppendStream(result, Row.class);
-		resultSet.addSink(new StreamITCase.StringSink<>());
+		StreamITCase.clear();
+		tEnv.toAppendStream(tEnv.sqlQuery("SELECT * FROM " + INPUT_TABLE), Row.class)
+			.addSink(new StreamITCase.StringSink<>());
 		env.execute();
 
-		List<String> expected = new ArrayList<>();
-		expected.add("1001");
-		expected.add("1002");
-		expected.add("1003");
-		expected.add("1004");
-		expected.add("1005");
-		expected.add("1006");
-		expected.add("1007");
-		expected.add("1008");
-		expected.add("1009");
-		expected.add("1010");
-
+		List<String> expected =
+			Arrays.asList(
+				"1,2020-01-01T15:35:00.123456,2020-01-01T15:35:00.123456789,15:35,1.175E-37,1.79769E308,100.1234",
+				"2,2020-01-01T15:36:01.123456,2020-01-01T15:36:01.123456789,15:36:01,-1.175E-37,-1.79769E308,101.1234");
 		StreamITCase.compareWithList(expected);
 	}
 
 	@Test
-	public void testAllFieldsSelection() throws Exception {
-		StreamITCase.clear();
+	public void testProjectableJDBCSource() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
+				.useBlinkPlanner()
+				.inStreamingMode()
+				.build();
+		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
+
+		tEnv.sqlUpdate(
+			"CREATE TABLE " + INPUT_TABLE + "(" +
+				"id BIGINT," +
+				"timestamp6_col TIMESTAMP(6)," +
+				"timestamp9_col TIMESTAMP(9)," +
+				"time_col TIME," +
+				"real_col FLOAT," +
+				"decimal_col DECIMAL(10, 4)" +
+				") WITH (" +
+				"  'connector.type'='jdbc'," +
+				"  'connector.url'='" + DB_URL + "'," +
+				"  'connector.table'='" + INPUT_TABLE + "'" +
+				")"
+		);
 
-		Table result = tEnv.sqlQuery(SELECT_ALL_BOOKS);
-		DataStream<Row> resultSet = tEnv.toAppendStream(result, Row.class);
-		resultSet.addSink(new StreamITCase.StringSink<>());
+		StreamITCase.clear();
+		tEnv.toAppendStream(tEnv.sqlQuery("SELECT timestamp6_col, decimal_col FROM " + INPUT_TABLE), Row.class)
+				.addSink(new StreamITCase.StringSink<>());
 		env.execute();
 
-		List<String> expected = new ArrayList<>();
-		expected.add("1001,Java public for dummies,Tan Ah Teck,11.11,11");
-		expected.add("1002,More Java for dummies,Tan Ah Teck,22.22,22");
-		expected.add("1003,More Java for more dummies,Mohammad Ali,33.33,33");
-		expected.add("1004,A Cup of Java,Kumar,44.44,44");
-		expected.add("1005,A Teaspoon of Java,Kevin Jones,55.55,55");
-		expected.add("1006,A Teaspoon of Java 1.4,Kevin Jones,66.66,66");
-		expected.add("1007,A Teaspoon of Java 1.5,Kevin Jones,77.77,77");
-		expected.add("1008,A Teaspoon of Java 1.6,Kevin Jones,88.88,88");
-		expected.add("1009,A Teaspoon of Java 1.7,Kevin Jones,99.99,99");
-		expected.add("1010,A Teaspoon of Java 1.8,Kevin Jones,null,1010");
-
+		List<String> expected =
+			Arrays.asList(
+				"2020-01-01T15:35:00.123456,100.1234",
+				"2020-01-01T15:36:01.123456,101.1234");
 		StreamITCase.compareWithList(expected);
 	}
 
+	@Test(expected = TableException.class)
+	public void testInvalidPrecisionOfJDBCSource() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
+				.useBlinkPlanner()
+				.inStreamingMode()
+				.build();
+		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
+
+		tEnv.sqlUpdate(
+		"CREATE TABLE " + INPUT_TABLE + "(" +
 
 Review comment:
   seems indent is not correct.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r377440708
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java
 ##########
 @@ -46,10 +59,84 @@
 		return Optional.empty();
 	}
 
-	private static class DerbyDialect implements JDBCDialect {
+	private abstract static class AbstractDialect implements JDBCDialect {
+
+		@Override
+		public void validate(TableSchema schema) throws ValidationException {
+			for (int i = 0; i < schema.getFieldCount(); i++) {
+				DataType dt = schema.getFieldDataType(i).get();
+				String fieldName = schema.getFieldName(i).get();
+
+				// TODO: We can't convert VARBINARY(n) data type to
+				//  PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO in LegacyTypeInfoDataTypeConverter
+				//  when n is smaller than Integer.MAX_VALUE
+				if (unsupportedTypes().contains(dt.getLogicalType().getTypeRoot()) ||
+						(!(dt.getLogicalType() instanceof LegacyTypeInformationType) &&
+						(VARBINARY == dt.getLogicalType().getTypeRoot()
+							&& Integer.MAX_VALUE != ((VarBinaryType) dt.getLogicalType()).getLength()))) {
+					throw new ValidationException(
+							String.format("The dialect don't support type: %s.", dt.toString()));
+				}
+
+				// only validate precision of DECIMAL type for blink planner
+				if (!(dt.getLogicalType() instanceof LegacyTypeInformationType)
+						&& DECIMAL == dt.getLogicalType().getTypeRoot()) {
+					int precision = ((DecimalType) dt.getLogicalType()).getPrecision();
+					if (precision > maxDecimalPrecision()
+							|| precision < minDecimalPrecision()) {
+						throw new ValidationException(
+								String.format("The precision of %s is out of the range [%d, %d].",
+										fieldName,
+										minDecimalPrecision(),
+										maxDecimalPrecision()));
+					}
+				}
+
+				// only validate precision of DECIMAL type for blink planner
+				if (!(dt.getLogicalType() instanceof LegacyTypeInformationType)
+						&& TIMESTAMP_WITHOUT_TIME_ZONE == dt.getLogicalType().getTypeRoot()) {
+					int precision = ((TimestampType) dt.getLogicalType()).getPrecision();
+					if (precision > maxTimestampPrecision()
+							|| precision < minTimestampPrecision()) {
+						throw new ValidationException(
+								String.format("The precision of %s is out of the range [%d, %d].",
 
 Review comment:
   The same to the DECIMAL type.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147345500 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148145921 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148156951 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   Hash:5e538c4875e6241d494585c6e2a8f586ac078131 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148370442 TriggerType:PUSH TriggerID:5e538c4875e6241d494585c6e2a8f586ac078131
   Hash:5e538c4875e6241d494585c6e2a8f586ac078131 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5056 TriggerType:PUSH TriggerID:5e538c4875e6241d494585c6e2a8f586ac078131
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   * f7456ac7e14681e18199941e0962b207eb99bef9 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/147345500) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813) 
   * 6336abbade5c71df3bd26fad40209e89642810d8 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148145921) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990) 
   * 330adf7ac3582db6b47a093e6f342351a3ba7e43 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148156951) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992) 
   * 5e538c4875e6241d494585c6e2a8f586ac078131 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148370442) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5056) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-585006962
 
 
   Waiting for the travis.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r362421728
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java
 ##########
 @@ -112,6 +113,11 @@ public boolean isBounded() {
 		return returnType;
 	}
 
+	@Override
+	public DataType getProducedDataType() {
+		return schema.toRowDataType();
 
 Review comment:
   updated

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376893098
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceITCase.java
 ##########
 @@ -18,93 +18,167 @@
 
 package org.apache.flink.api.java.io.jdbc;
 
-import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.api.java.StreamTableEnvironment;
 import org.apache.flink.table.runtime.utils.StreamITCase;
+import org.apache.flink.test.util.AbstractTestBase;
 import org.apache.flink.types.Row;
 
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
-import java.util.ArrayList;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
 import java.util.List;
 
+
 /**
- * IT case for {@link JDBCTableSource}.
+ * ITCase for {@link JDBCTableSource}.
  */
-public class JDBCTableSourceITCase extends JDBCTestBase {
-
-	private static final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-	private static final EnvironmentSettings bsSettings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build();
-	private static final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, bsSettings);
-
-	static final String TABLE_SOURCE_SQL = "CREATE TABLE books (" +
-		" id int, " +
-		" title varchar, " +
-		" author varchar, " +
-		" price double, " +
-		" qty int " +
-		") with (" +
-		" 'connector.type' = 'jdbc', " +
-		" 'connector.url' = 'jdbc:derby:memory:ebookshop', " +
-		" 'connector.table' = 'books', " +
-		" 'connector.driver' = 'org.apache.derby.jdbc.EmbeddedDriver' " +
-		")";
-
-	@BeforeClass
-	public static void createTable() {
-		tEnv.sqlUpdate(TABLE_SOURCE_SQL);
+public class JDBCTableSourceITCase extends AbstractTestBase {
+
+	public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver";
+	public static final String DB_URL = "jdbc:derby:memory:test";
+	public static final String INPUT_TABLE = "jdbcSource";
+
+	@Before
+	public void before() throws ClassNotFoundException, SQLException {
+		System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL");
+		Class.forName(DRIVER_CLASS);
+
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL + ";create=true");
+			Statement statement = conn.createStatement()) {
+			statement.executeUpdate("CREATE TABLE " + INPUT_TABLE + " (" +
+					"id BIGINT NOT NULL," +
+					"timestamp6_col TIMESTAMP, " +
+					"timestamp9_col TIMESTAMP, " +
+					"time_col TIME, " +
+					"real_col FLOAT(23), " +    // A precision of 23 or less makes FLOAT equivalent to REAL.
+					"double_col FLOAT(24)," +   // A precision of 24 or greater makes FLOAT equivalent to DOUBLE PRECISION.
+					"decimal_col DECIMAL(10, 4))");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"1, TIMESTAMP('2020-01-01 15:35:00.123456'), TIMESTAMP('2020-01-01 15:35:00.123456789'), " +
+					"TIME('15:35:00'), 1.175E-37, 1.79769E+308, 100.1234)");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"2, TIMESTAMP('2020-01-01 15:36:01.123456'), TIMESTAMP('2020-01-01 15:36:01.123456789'), " +
+					"TIME('15:36:01'), -1.175E-37, -1.79769E+308, 101.1234)");
+		}
+	}
+
+	@After
+	public void clearOutputTable() throws Exception {
+		Class.forName(DRIVER_CLASS);
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL);
+			Statement stat = conn.createStatement()) {
+			stat.execute("DROP TABLE " + INPUT_TABLE);
 
 Review comment:
   execute can be used for any SQL statement, but executeUpdate is more clearly here. I will fix this.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r377619154
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java
 ##########
 @@ -71,43 +71,48 @@ public void validate(TableSchema schema) throws ValidationException {
 				//  PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO in LegacyTypeInfoDataTypeConverter
 				//  when n is smaller than Integer.MAX_VALUE
 				if (unsupportedTypes().contains(dt.getLogicalType().getTypeRoot()) ||
-						(!(dt.getLogicalType() instanceof LegacyTypeInformationType) &&
-						(VARBINARY == dt.getLogicalType().getTypeRoot()
-							&& Integer.MAX_VALUE != ((VarBinaryType) dt.getLogicalType()).getLength()))) {
+						(dt.getLogicalType() instanceof VarBinaryType
+							&& Integer.MAX_VALUE != ((VarBinaryType) dt.getLogicalType()).getLength())) {
 					throw new ValidationException(
-							String.format("The dialect don't support type: %s.", dt.toString()));
+							String.format("The %s dialect don't support type: %s.",
 
 Review comment:
   Nit: don't -> doesn't

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/147345500 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   * f7456ac7e14681e18199941e0962b207eb99bef9 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/147345500) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147345500 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148145921 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148156951 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   Hash:5e538c4875e6241d494585c6e2a8f586ac078131 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148370442 TriggerType:PUSH TriggerID:5e538c4875e6241d494585c6e2a8f586ac078131
   Hash:5e538c4875e6241d494585c6e2a8f586ac078131 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5056 TriggerType:PUSH TriggerID:5e538c4875e6241d494585c6e2a8f586ac078131
   Hash:7790797e5e90af02bc94b48f5326077573600f25 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5082 TriggerType:PUSH TriggerID:7790797e5e90af02bc94b48f5326077573600f25
   Hash:7790797e5e90af02bc94b48f5326077573600f25 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148511660 TriggerType:PUSH TriggerID:7790797e5e90af02bc94b48f5326077573600f25
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   * f7456ac7e14681e18199941e0962b207eb99bef9 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/147345500) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813) 
   * 6336abbade5c71df3bd26fad40209e89642810d8 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148145921) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990) 
   * 330adf7ac3582db6b47a093e6f342351a3ba7e43 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148156951) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992) 
   * 5e538c4875e6241d494585c6e2a8f586ac078131 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148370442) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5056) 
   * 7790797e5e90af02bc94b48f5326077573600f25 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148511660) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5082) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376202505
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialect.java
 ##########
 @@ -35,6 +37,15 @@
 	 */
 	boolean canHandle(String url);
 
+	/**
+	 * Check if this dialect instance support a specific data type in table schema.
+	 *
+	 * @param schema the table schema
+	 */
+	default void validate(TableSchema schema) {
+		return;
 
 Review comment:
   We don't need to `return` if the return type is `void`.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r377450437
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceITCase.java
 ##########
 @@ -18,93 +18,167 @@
 
 package org.apache.flink.api.java.io.jdbc;
 
-import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.api.java.StreamTableEnvironment;
 import org.apache.flink.table.runtime.utils.StreamITCase;
+import org.apache.flink.test.util.AbstractTestBase;
 import org.apache.flink.types.Row;
 
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
-import java.util.ArrayList;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
 import java.util.List;
 
+
 /**
- * IT case for {@link JDBCTableSource}.
+ * ITCase for {@link JDBCTableSource}.
  */
-public class JDBCTableSourceITCase extends JDBCTestBase {
-
-	private static final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-	private static final EnvironmentSettings bsSettings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build();
-	private static final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, bsSettings);
-
-	static final String TABLE_SOURCE_SQL = "CREATE TABLE books (" +
-		" id int, " +
-		" title varchar, " +
-		" author varchar, " +
-		" price double, " +
-		" qty int " +
-		") with (" +
-		" 'connector.type' = 'jdbc', " +
-		" 'connector.url' = 'jdbc:derby:memory:ebookshop', " +
-		" 'connector.table' = 'books', " +
-		" 'connector.driver' = 'org.apache.derby.jdbc.EmbeddedDriver' " +
-		")";
-
-	@BeforeClass
-	public static void createTable() {
-		tEnv.sqlUpdate(TABLE_SOURCE_SQL);
+public class JDBCTableSourceITCase extends AbstractTestBase {
+
+	public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver";
+	public static final String DB_URL = "jdbc:derby:memory:test";
+	public static final String INPUT_TABLE = "jdbcSource";
+
+	@Before
+	public void before() throws ClassNotFoundException, SQLException {
+		System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL");
+		Class.forName(DRIVER_CLASS);
+
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL + ";create=true");
+			Statement statement = conn.createStatement()) {
+			statement.executeUpdate("CREATE TABLE " + INPUT_TABLE + " (" +
+					"id BIGINT NOT NULL," +
+					"timestamp6_col TIMESTAMP, " +
+					"timestamp9_col TIMESTAMP, " +
+					"time_col TIME, " +
+					"real_col FLOAT(23), " +    // A precision of 23 or less makes FLOAT equivalent to REAL.
+					"double_col FLOAT(24)," +   // A precision of 24 or greater makes FLOAT equivalent to DOUBLE PRECISION.
+					"decimal_col DECIMAL(10, 4))");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"1, TIMESTAMP('2020-01-01 15:35:00.123456'), TIMESTAMP('2020-01-01 15:35:00.123456789'), " +
+					"TIME('15:35:00'), 1.175E-37, 1.79769E+308, 100.1234)");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"2, TIMESTAMP('2020-01-01 15:36:01.123456'), TIMESTAMP('2020-01-01 15:36:01.123456789'), " +
+					"TIME('15:36:01'), -1.175E-37, -1.79769E+308, 101.1234)");
+		}
+	}
+
+	@After
+	public void clearOutputTable() throws Exception {
+		Class.forName(DRIVER_CLASS);
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL);
+			Statement stat = conn.createStatement()) {
+			stat.execute("DROP TABLE " + INPUT_TABLE);
+		}
 	}
 
 	@Test
-	public void testFieldsProjection() throws Exception {
-		StreamITCase.clear();
+	public void testJDBCSource() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
+			.useBlinkPlanner()
+			.inStreamingMode()
+			.build();
+		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
+
+		tEnv.sqlUpdate(
+			"CREATE TABLE " + INPUT_TABLE + "(" +
+				"id BIGINT," +
+				"timestamp6_col TIMESTAMP(6)," +
+				"timestamp9_col TIMESTAMP(9)," +
+				"time_col TIME," +
+				"real_col FLOAT," +
+				"double_col DOUBLE," +
+				"decimal_col DECIMAL(10, 4)" +
+				") WITH (" +
+				"  'connector.type'='jdbc'," +
+				"  'connector.url'='" + DB_URL + "'," +
+				"  'connector.table'='" + INPUT_TABLE + "'" +
+				")"
+		);
 
-		Table result = tEnv.sqlQuery(SELECT_ID_BOOKS);
-		DataStream<Row> resultSet = tEnv.toAppendStream(result, Row.class);
-		resultSet.addSink(new StreamITCase.StringSink<>());
+		StreamITCase.clear();
+		tEnv.toAppendStream(tEnv.sqlQuery("SELECT * FROM " + INPUT_TABLE), Row.class)
+			.addSink(new StreamITCase.StringSink<>());
 		env.execute();
 
-		List<String> expected = new ArrayList<>();
-		expected.add("1001");
-		expected.add("1002");
-		expected.add("1003");
-		expected.add("1004");
-		expected.add("1005");
-		expected.add("1006");
-		expected.add("1007");
-		expected.add("1008");
-		expected.add("1009");
-		expected.add("1010");
-
+		List<String> expected =
+			Arrays.asList(
+				"1,2020-01-01T15:35:00.123456,2020-01-01T15:35:00.123456789,15:35,1.175E-37,1.79769E308,100.1234",
+				"2,2020-01-01T15:36:01.123456,2020-01-01T15:36:01.123456789,15:36:01,-1.175E-37,-1.79769E308,101.1234");
 		StreamITCase.compareWithList(expected);
 	}
 
 	@Test
-	public void testAllFieldsSelection() throws Exception {
-		StreamITCase.clear();
+	public void testProjectableJDBCSource() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
+				.useBlinkPlanner()
+				.inStreamingMode()
+				.build();
+		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
+
+		tEnv.sqlUpdate(
+			"CREATE TABLE " + INPUT_TABLE + "(" +
+				"id BIGINT," +
+				"timestamp6_col TIMESTAMP(6)," +
+				"timestamp9_col TIMESTAMP(9)," +
+				"time_col TIME," +
+				"real_col FLOAT," +
+				"decimal_col DECIMAL(10, 4)" +
+				") WITH (" +
+				"  'connector.type'='jdbc'," +
+				"  'connector.url'='" + DB_URL + "'," +
+				"  'connector.table'='" + INPUT_TABLE + "'" +
+				")"
+		);
 
-		Table result = tEnv.sqlQuery(SELECT_ALL_BOOKS);
-		DataStream<Row> resultSet = tEnv.toAppendStream(result, Row.class);
-		resultSet.addSink(new StreamITCase.StringSink<>());
+		StreamITCase.clear();
+		tEnv.toAppendStream(tEnv.sqlQuery("SELECT timestamp6_col, decimal_col FROM " + INPUT_TABLE), Row.class)
+				.addSink(new StreamITCase.StringSink<>());
 		env.execute();
 
-		List<String> expected = new ArrayList<>();
-		expected.add("1001,Java public for dummies,Tan Ah Teck,11.11,11");
-		expected.add("1002,More Java for dummies,Tan Ah Teck,22.22,22");
-		expected.add("1003,More Java for more dummies,Mohammad Ali,33.33,33");
-		expected.add("1004,A Cup of Java,Kumar,44.44,44");
-		expected.add("1005,A Teaspoon of Java,Kevin Jones,55.55,55");
-		expected.add("1006,A Teaspoon of Java 1.4,Kevin Jones,66.66,66");
-		expected.add("1007,A Teaspoon of Java 1.5,Kevin Jones,77.77,77");
-		expected.add("1008,A Teaspoon of Java 1.6,Kevin Jones,88.88,88");
-		expected.add("1009,A Teaspoon of Java 1.7,Kevin Jones,99.99,99");
-		expected.add("1010,A Teaspoon of Java 1.8,Kevin Jones,null,1010");
-
+		List<String> expected =
+			Arrays.asList(
+				"2020-01-01T15:35:00.123456,100.1234",
+				"2020-01-01T15:36:01.123456,101.1234");
 		StreamITCase.compareWithList(expected);
 	}
 
+	@Test(expected = TableException.class)
+	public void testInvalidPrecisionOfJDBCSource() throws Exception {
 
 Review comment:
   I think we don't need this test any more, because is already covered by `JDBCDataTypeTest`.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] libenchao commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
libenchao commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376680607
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialect.java
 ##########
 @@ -35,6 +37,15 @@
 	 */
 	boolean canHandle(String url);
 
+	/**
+	 * Check if this dialect instance support a specific data type in table schema.
+	 *
+	 * @param schema the table schema
+	 */
+	default void validate(TableSchema schema) {
 
 Review comment:
   `ValidationException` is a `RuntimeException`, adding it to method signature won' t force caller the check it. Maybe we just need to add it in java doc like `LogicalTypeParser.parse` and `FieldInfoUtils.validateInputTypeInfo` ?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r362691371
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java
 ##########
 @@ -112,6 +120,11 @@ public boolean isBounded() {
 		return returnType;
 	}
 
+	@Override
+	public DataType getProducedDataType() {
 
 Review comment:
   Sure
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376199098
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialect.java
 ##########
 @@ -35,6 +37,15 @@
 	 */
 	boolean canHandle(String url);
 
+	/**
+	 * Check if this dialect instance support a specific data type in table schema.
+	 *
 
 Review comment:
   remove emtpy line.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147345500 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   * f7456ac7e14681e18199941e0962b207eb99bef9 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/147345500) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] docete commented on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
docete commented on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-583988903
 
 
   @wuchong pls take another look. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r377450538
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java
 ##########
 @@ -46,10 +59,84 @@
 		return Optional.empty();
 	}
 
-	private static class DerbyDialect implements JDBCDialect {
+	private abstract static class AbstractDialect implements JDBCDialect {
+
+		@Override
+		public void validate(TableSchema schema) throws ValidationException {
+			for (int i = 0; i < schema.getFieldCount(); i++) {
+				DataType dt = schema.getFieldDataType(i).get();
+				String fieldName = schema.getFieldName(i).get();
+
+				// TODO: We can't convert VARBINARY(n) data type to
+				//  PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO in LegacyTypeInfoDataTypeConverter
+				//  when n is smaller than Integer.MAX_VALUE
+				if (unsupportedTypes().contains(dt.getLogicalType().getTypeRoot()) ||
+						(!(dt.getLogicalType() instanceof LegacyTypeInformationType) &&
+						(VARBINARY == dt.getLogicalType().getTypeRoot()
+							&& Integer.MAX_VALUE != ((VarBinaryType) dt.getLogicalType()).getLength()))) {
+					throw new ValidationException(
+							String.format("The dialect don't support type: %s.", dt.toString()));
 
 Review comment:
   String.format("The %s dialect doesn't support type: %s.", dialectName, dt.toString())

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147345500 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   * f7456ac7e14681e18199941e0962b207eb99bef9 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/147345500) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813) 
   * 6336abbade5c71df3bd26fad40209e89642810d8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147345500 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148145921 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148156951 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   * f7456ac7e14681e18199941e0962b207eb99bef9 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/147345500) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813) 
   * 6336abbade5c71df3bd26fad40209e89642810d8 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148145921) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990) 
   * 330adf7ac3582db6b47a093e6f342351a3ba7e43 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148156951) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r362691385
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java
 ##########
 @@ -180,6 +193,20 @@ public boolean equals(Object o) {
 		}
 	}
 
+	@Override
+	public String explainSource() {
+		if (selectFields == null) {
+			return String.format(
+				"JDBCTableSource(read fields: %s)", String.join(", ", schema.getFieldNames()));
+		} else {
+			String[] fields = new String[selectFields.length];
+			for (int i = 0; i < selectFields.length; i++) {
+				fields[i] = schema.getFieldName(selectFields[i]).get();
+			}
+			return String.format("JDBCTableSource(read fields: %s)", String.join(", ", fields));
 
 Review comment:
   Sure

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376266784
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java
 ##########
 @@ -73,17 +75,23 @@ private JDBCTableSource(
 		this.selectFields = selectFields;
 
 		final TypeInformation<?>[] schemaTypeInfos = schema.getFieldTypes();
+		final DataType[] schemaDataTypes = schema.getFieldDataTypes();
 		final String[] schemaFieldNames = schema.getFieldNames();
 		if (selectFields != null) {
 			TypeInformation<?>[] typeInfos = new TypeInformation[selectFields.length];
-			String[] typeNames = new String[selectFields.length];
+			DataType[] dataTypes = new DataType[selectFields.length];
+			String[] fieldNames = new String[selectFields.length];
 			for (int i = 0; i < selectFields.length; i++) {
 				typeInfos[i] = schemaTypeInfos[selectFields[i]];
-				typeNames[i] = schemaFieldNames[selectFields[i]];
+				dataTypes[i] = schemaDataTypes[selectFields[i]];
+				fieldNames[i] = schemaFieldNames[selectFields[i]];
 			}
-			this.returnType = new RowTypeInfo(typeInfos, typeNames);
+			this.returnType = new RowTypeInfo(typeInfos, fieldNames);
 
 Review comment:
   Make sense. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147345500 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148145921 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148156951 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   Hash:5e538c4875e6241d494585c6e2a8f586ac078131 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:5e538c4875e6241d494585c6e2a8f586ac078131
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   * f7456ac7e14681e18199941e0962b207eb99bef9 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/147345500) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813) 
   * 6336abbade5c71df3bd26fad40209e89642810d8 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148145921) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990) 
   * 330adf7ac3582db6b47a093e6f342351a3ba7e43 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148156951) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992) 
   * 5e538c4875e6241d494585c6e2a8f586ac078131 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r362465428
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCTableSource.java
 ##########
 @@ -180,6 +193,20 @@ public boolean equals(Object o) {
 		}
 	}
 
+	@Override
+	public String explainSource() {
+		if (selectFields == null) {
+			return String.format(
+				"JDBCTableSource(read fields: %s)", String.join(", ", schema.getFieldNames()));
+		} else {
+			String[] fields = new String[selectFields.length];
+			for (int i = 0; i < selectFields.length; i++) {
+				fields[i] = schema.getFieldName(selectFields[i]).get();
+			}
+			return String.format("JDBCTableSource(read fields: %s)", String.join(", ", fields));
 
 Review comment:
   Use `returnType.getFieldNames` instead reconstruct the selected field names again?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r377440632
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/dialect/JDBCDialects.java
 ##########
 @@ -46,10 +59,84 @@
 		return Optional.empty();
 	}
 
-	private static class DerbyDialect implements JDBCDialect {
+	private abstract static class AbstractDialect implements JDBCDialect {
+
+		@Override
+		public void validate(TableSchema schema) throws ValidationException {
+			for (int i = 0; i < schema.getFieldCount(); i++) {
+				DataType dt = schema.getFieldDataType(i).get();
+				String fieldName = schema.getFieldName(i).get();
+
+				// TODO: We can't convert VARBINARY(n) data type to
+				//  PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO in LegacyTypeInfoDataTypeConverter
+				//  when n is smaller than Integer.MAX_VALUE
+				if (unsupportedTypes().contains(dt.getLogicalType().getTypeRoot()) ||
+						(!(dt.getLogicalType() instanceof LegacyTypeInformationType) &&
+						(VARBINARY == dt.getLogicalType().getTypeRoot()
+							&& Integer.MAX_VALUE != ((VarBinaryType) dt.getLogicalType()).getLength()))) {
+					throw new ValidationException(
+							String.format("The dialect don't support type: %s.", dt.toString()));
+				}
+
+				// only validate precision of DECIMAL type for blink planner
+				if (!(dt.getLogicalType() instanceof LegacyTypeInformationType)
+						&& DECIMAL == dt.getLogicalType().getTypeRoot()) {
+					int precision = ((DecimalType) dt.getLogicalType()).getPrecision();
+					if (precision > maxDecimalPrecision()
+							|| precision < minDecimalPrecision()) {
+						throw new ValidationException(
+								String.format("The precision of %s is out of the range [%d, %d].",
+										fieldName,
+										minDecimalPrecision(),
+										maxDecimalPrecision()));
+					}
+				}
+
+				// only validate precision of DECIMAL type for blink planner
+				if (!(dt.getLogicalType() instanceof LegacyTypeInformationType)
+						&& TIMESTAMP_WITHOUT_TIME_ZONE == dt.getLogicalType().getTypeRoot()) {
+					int precision = ((TimestampType) dt.getLogicalType()).getPrecision();
+					if (precision > maxTimestampPrecision()
+							|| precision < minTimestampPrecision()) {
+						throw new ValidationException(
+								String.format("The precision of %s is out of the range [%d, %d].",
 
 Review comment:
   Improve the error message a bit more:
   
   ```java
   String.format("The precision of filed '%s' is out of the TIMESTAMP precision range [%d, %d] supported by the %s dialect.",
   fieldName,
   minTimestampPrecision(),
   maxTimestampPrecision(),
   dialectName);
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong merged pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong merged pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745
 
 
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147345500 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148145921 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148156951 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   Hash:5e538c4875e6241d494585c6e2a8f586ac078131 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148370442 TriggerType:PUSH TriggerID:5e538c4875e6241d494585c6e2a8f586ac078131
   Hash:5e538c4875e6241d494585c6e2a8f586ac078131 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5056 TriggerType:PUSH TriggerID:5e538c4875e6241d494585c6e2a8f586ac078131
   Hash:7790797e5e90af02bc94b48f5326077573600f25 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:7790797e5e90af02bc94b48f5326077573600f25
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   * f7456ac7e14681e18199941e0962b207eb99bef9 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/147345500) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813) 
   * 6336abbade5c71df3bd26fad40209e89642810d8 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148145921) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990) 
   * 330adf7ac3582db6b47a093e6f342351a3ba7e43 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148156951) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992) 
   * 5e538c4875e6241d494585c6e2a8f586ac078131 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148370442) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5056) 
   * 7790797e5e90af02bc94b48f5326077573600f25 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147345500 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148145921 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148156951 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992 TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   Hash:5e538c4875e6241d494585c6e2a8f586ac078131 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148370442 TriggerType:PUSH TriggerID:5e538c4875e6241d494585c6e2a8f586ac078131
   Hash:5e538c4875e6241d494585c6e2a8f586ac078131 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5056 TriggerType:PUSH TriggerID:5e538c4875e6241d494585c6e2a8f586ac078131
   Hash:7790797e5e90af02bc94b48f5326077573600f25 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5082 TriggerType:PUSH TriggerID:7790797e5e90af02bc94b48f5326077573600f25
   Hash:7790797e5e90af02bc94b48f5326077573600f25 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/148511660 TriggerType:PUSH TriggerID:7790797e5e90af02bc94b48f5326077573600f25
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   * f7456ac7e14681e18199941e0962b207eb99bef9 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/147345500) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813) 
   * 6336abbade5c71df3bd26fad40209e89642810d8 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148145921) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990) 
   * 330adf7ac3582db6b47a093e6f342351a3ba7e43 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148156951) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4992) 
   * 5e538c4875e6241d494585c6e2a8f586ac078131 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148370442) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5056) 
   * 7790797e5e90af02bc94b48f5326077573600f25 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/148511660) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5082) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
wuchong commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376198345
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceITCase.java
 ##########
 @@ -18,93 +18,167 @@
 
 package org.apache.flink.api.java.io.jdbc;
 
-import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.api.java.StreamTableEnvironment;
 import org.apache.flink.table.runtime.utils.StreamITCase;
+import org.apache.flink.test.util.AbstractTestBase;
 import org.apache.flink.types.Row;
 
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
-import java.util.ArrayList;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
 import java.util.List;
 
+
 /**
- * IT case for {@link JDBCTableSource}.
+ * ITCase for {@link JDBCTableSource}.
  */
-public class JDBCTableSourceITCase extends JDBCTestBase {
-
-	private static final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-	private static final EnvironmentSettings bsSettings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build();
-	private static final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, bsSettings);
-
-	static final String TABLE_SOURCE_SQL = "CREATE TABLE books (" +
-		" id int, " +
-		" title varchar, " +
-		" author varchar, " +
-		" price double, " +
-		" qty int " +
-		") with (" +
-		" 'connector.type' = 'jdbc', " +
-		" 'connector.url' = 'jdbc:derby:memory:ebookshop', " +
-		" 'connector.table' = 'books', " +
-		" 'connector.driver' = 'org.apache.derby.jdbc.EmbeddedDriver' " +
-		")";
-
-	@BeforeClass
-	public static void createTable() {
-		tEnv.sqlUpdate(TABLE_SOURCE_SQL);
+public class JDBCTableSourceITCase extends AbstractTestBase {
+
+	public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver";
+	public static final String DB_URL = "jdbc:derby:memory:test";
+	public static final String INPUT_TABLE = "jdbcSource";
+
+	@Before
+	public void before() throws ClassNotFoundException, SQLException {
+		System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL");
+		Class.forName(DRIVER_CLASS);
+
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL + ";create=true");
+			Statement statement = conn.createStatement()) {
+			statement.executeUpdate("CREATE TABLE " + INPUT_TABLE + " (" +
+					"id BIGINT NOT NULL," +
+					"timestamp6_col TIMESTAMP, " +
+					"timestamp9_col TIMESTAMP, " +
+					"time_col TIME, " +
+					"real_col FLOAT(23), " +    // A precision of 23 or less makes FLOAT equivalent to REAL.
+					"double_col FLOAT(24)," +   // A precision of 24 or greater makes FLOAT equivalent to DOUBLE PRECISION.
+					"decimal_col DECIMAL(10, 4))");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"1, TIMESTAMP('2020-01-01 15:35:00.123456'), TIMESTAMP('2020-01-01 15:35:00.123456789'), " +
+					"TIME('15:35:00'), 1.175E-37, 1.79769E+308, 100.1234)");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"2, TIMESTAMP('2020-01-01 15:36:01.123456'), TIMESTAMP('2020-01-01 15:36:01.123456789'), " +
+					"TIME('15:36:01'), -1.175E-37, -1.79769E+308, 101.1234)");
+		}
+	}
+
+	@After
+	public void clearOutputTable() throws Exception {
+		Class.forName(DRIVER_CLASS);
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL);
+			Statement stat = conn.createStatement()) {
+			stat.execute("DROP TABLE " + INPUT_TABLE);
+		}
 	}
 
 	@Test
-	public void testFieldsProjection() throws Exception {
-		StreamITCase.clear();
+	public void testJDBCSource() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
+			.useBlinkPlanner()
+			.inStreamingMode()
+			.build();
+		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
+
+		tEnv.sqlUpdate(
+			"CREATE TABLE " + INPUT_TABLE + "(" +
+				"id BIGINT," +
+				"timestamp6_col TIMESTAMP(6)," +
+				"timestamp9_col TIMESTAMP(9)," +
+				"time_col TIME," +
+				"real_col FLOAT," +
+				"double_col DOUBLE," +
+				"decimal_col DECIMAL(10, 4)" +
+				") WITH (" +
+				"  'connector.type'='jdbc'," +
+				"  'connector.url'='" + DB_URL + "'," +
+				"  'connector.table'='" + INPUT_TABLE + "'" +
+				")"
+		);
 
-		Table result = tEnv.sqlQuery(SELECT_ID_BOOKS);
-		DataStream<Row> resultSet = tEnv.toAppendStream(result, Row.class);
-		resultSet.addSink(new StreamITCase.StringSink<>());
+		StreamITCase.clear();
+		tEnv.toAppendStream(tEnv.sqlQuery("SELECT * FROM " + INPUT_TABLE), Row.class)
+			.addSink(new StreamITCase.StringSink<>());
 		env.execute();
 
-		List<String> expected = new ArrayList<>();
-		expected.add("1001");
-		expected.add("1002");
-		expected.add("1003");
-		expected.add("1004");
-		expected.add("1005");
-		expected.add("1006");
-		expected.add("1007");
-		expected.add("1008");
-		expected.add("1009");
-		expected.add("1010");
-
+		List<String> expected =
+			Arrays.asList(
+				"1,2020-01-01T15:35:00.123456,2020-01-01T15:35:00.123456789,15:35,1.175E-37,1.79769E308,100.1234",
+				"2,2020-01-01T15:36:01.123456,2020-01-01T15:36:01.123456789,15:36:01,-1.175E-37,-1.79769E308,101.1234");
 		StreamITCase.compareWithList(expected);
 	}
 
 	@Test
-	public void testAllFieldsSelection() throws Exception {
-		StreamITCase.clear();
+	public void testProjectableJDBCSource() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
+				.useBlinkPlanner()
+				.inStreamingMode()
+				.build();
+		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
+
+		tEnv.sqlUpdate(
+			"CREATE TABLE " + INPUT_TABLE + "(" +
+				"id BIGINT," +
+				"timestamp6_col TIMESTAMP(6)," +
+				"timestamp9_col TIMESTAMP(9)," +
+				"time_col TIME," +
+				"real_col FLOAT," +
+				"decimal_col DECIMAL(10, 4)" +
+				") WITH (" +
+				"  'connector.type'='jdbc'," +
+				"  'connector.url'='" + DB_URL + "'," +
+				"  'connector.table'='" + INPUT_TABLE + "'" +
+				")"
+		);
 
-		Table result = tEnv.sqlQuery(SELECT_ALL_BOOKS);
-		DataStream<Row> resultSet = tEnv.toAppendStream(result, Row.class);
-		resultSet.addSink(new StreamITCase.StringSink<>());
+		StreamITCase.clear();
+		tEnv.toAppendStream(tEnv.sqlQuery("SELECT timestamp6_col, decimal_col FROM " + INPUT_TABLE), Row.class)
+				.addSink(new StreamITCase.StringSink<>());
 		env.execute();
 
-		List<String> expected = new ArrayList<>();
-		expected.add("1001,Java public for dummies,Tan Ah Teck,11.11,11");
-		expected.add("1002,More Java for dummies,Tan Ah Teck,22.22,22");
-		expected.add("1003,More Java for more dummies,Mohammad Ali,33.33,33");
-		expected.add("1004,A Cup of Java,Kumar,44.44,44");
-		expected.add("1005,A Teaspoon of Java,Kevin Jones,55.55,55");
-		expected.add("1006,A Teaspoon of Java 1.4,Kevin Jones,66.66,66");
-		expected.add("1007,A Teaspoon of Java 1.5,Kevin Jones,77.77,77");
-		expected.add("1008,A Teaspoon of Java 1.6,Kevin Jones,88.88,88");
-		expected.add("1009,A Teaspoon of Java 1.7,Kevin Jones,99.99,99");
-		expected.add("1010,A Teaspoon of Java 1.8,Kevin Jones,null,1010");
-
+		List<String> expected =
+			Arrays.asList(
+				"2020-01-01T15:35:00.123456,100.1234",
+				"2020-01-01T15:36:01.123456,101.1234");
 		StreamITCase.compareWithList(expected);
 	}
 
+	@Test(expected = TableException.class)
+	public void testInvalidPrecisionOfJDBCSource() throws Exception {
 
 Review comment:
   You can create a unit test `JDBCDataTypeTest` to verify all the types with different precision with different dialects. This doesn't involve a job submission, and is a lightweight unit test. You can take `FlinkDDLDataTypeTest` as an example.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147345500 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148145921 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   * f7456ac7e14681e18199941e0962b207eb99bef9 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/147345500) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813) 
   * 6336abbade5c71df3bd26fad40209e89642810d8 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148145921) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#issuecomment-570165880
 
 
   <!--
   Meta data
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/142855689 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:862f3c5fd9e77184af08aea1231cf3a00e03b193 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042 TriggerType:PUSH TriggerID:862f3c5fd9e77184af08aea1231cf3a00e03b193
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/142858894 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Status:PENDING URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044 TriggerType:PUSH TriggerID:2cc4e7b6e88e9f655235b8c97707c41f45b6b238
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/142952111 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:67d02dcc57087af3626bcbc5cd1cd51117de2b83 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053 TriggerType:PUSH TriggerID:67d02dcc57087af3626bcbc5cd1cd51117de2b83
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/143225362 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:616c1a840307eeefb7fa7fa970d65a149ff7af99 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123 TriggerType:PUSH TriggerID:616c1a840307eeefb7fa7fa970d65a149ff7af99
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/143885991 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:30d7534882a4c84d906dce5d81e9807cce6c051d Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257 TriggerType:PUSH TriggerID:30d7534882a4c84d906dce5d81e9807cce6c051d
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/144085481 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:d915cd1e7d4efc5a2fb114be21483e249f2eca0b Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276 TriggerType:PUSH TriggerID:d915cd1e7d4efc5a2fb114be21483e249f2eca0b
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/147345500 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:f7456ac7e14681e18199941e0962b207eb99bef9 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813 TriggerType:PUSH TriggerID:f7456ac7e14681e18199941e0962b207eb99bef9
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/148145921 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:6336abbade5c71df3bd26fad40209e89642810d8 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990 TriggerType:PUSH TriggerID:6336abbade5c71df3bd26fad40209e89642810d8
   Hash:330adf7ac3582db6b47a093e6f342351a3ba7e43 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:330adf7ac3582db6b47a093e6f342351a3ba7e43
   -->
   ## CI report:
   
   * 862f3c5fd9e77184af08aea1231cf3a00e03b193 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/142855689) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4042) 
   * 2cc4e7b6e88e9f655235b8c97707c41f45b6b238 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/142858894) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4044) 
   * 67d02dcc57087af3626bcbc5cd1cd51117de2b83 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/142952111) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4053) 
   * 616c1a840307eeefb7fa7fa970d65a149ff7af99 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/143225362) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4123) 
   * 30d7534882a4c84d906dce5d81e9807cce6c051d Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/143885991) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4257) 
   * d915cd1e7d4efc5a2fb114be21483e249f2eca0b Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/144085481) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4276) 
   * f7456ac7e14681e18199941e0962b207eb99bef9 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/147345500) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4813) 
   * 6336abbade5c71df3bd26fad40209e89642810d8 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/148145921) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=4990) 
   * 330adf7ac3582db6b47a093e6f342351a3ba7e43 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…

Posted by GitBox <gi...@apache.org>.
docete commented on a change in pull request #10745: [FLINK-15445][connectors/jdbc] JDBC Table Source didn't work for Type…
URL: https://github.com/apache/flink/pull/10745#discussion_r376897708
 
 

 ##########
 File path: flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceITCase.java
 ##########
 @@ -18,93 +18,167 @@
 
 package org.apache.flink.api.java.io.jdbc;
 
-import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.api.java.StreamTableEnvironment;
 import org.apache.flink.table.runtime.utils.StreamITCase;
+import org.apache.flink.test.util.AbstractTestBase;
 import org.apache.flink.types.Row;
 
-import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
-import java.util.ArrayList;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
 import java.util.List;
 
+
 /**
- * IT case for {@link JDBCTableSource}.
+ * ITCase for {@link JDBCTableSource}.
  */
-public class JDBCTableSourceITCase extends JDBCTestBase {
-
-	private static final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-	private static final EnvironmentSettings bsSettings = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build();
-	private static final StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, bsSettings);
-
-	static final String TABLE_SOURCE_SQL = "CREATE TABLE books (" +
-		" id int, " +
-		" title varchar, " +
-		" author varchar, " +
-		" price double, " +
-		" qty int " +
-		") with (" +
-		" 'connector.type' = 'jdbc', " +
-		" 'connector.url' = 'jdbc:derby:memory:ebookshop', " +
-		" 'connector.table' = 'books', " +
-		" 'connector.driver' = 'org.apache.derby.jdbc.EmbeddedDriver' " +
-		")";
-
-	@BeforeClass
-	public static void createTable() {
-		tEnv.sqlUpdate(TABLE_SOURCE_SQL);
+public class JDBCTableSourceITCase extends AbstractTestBase {
+
+	public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver";
+	public static final String DB_URL = "jdbc:derby:memory:test";
+	public static final String INPUT_TABLE = "jdbcSource";
+
+	@Before
+	public void before() throws ClassNotFoundException, SQLException {
+		System.setProperty("derby.stream.error.field", JDBCTestBase.class.getCanonicalName() + ".DEV_NULL");
+		Class.forName(DRIVER_CLASS);
+
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL + ";create=true");
+			Statement statement = conn.createStatement()) {
+			statement.executeUpdate("CREATE TABLE " + INPUT_TABLE + " (" +
+					"id BIGINT NOT NULL," +
+					"timestamp6_col TIMESTAMP, " +
+					"timestamp9_col TIMESTAMP, " +
+					"time_col TIME, " +
+					"real_col FLOAT(23), " +    // A precision of 23 or less makes FLOAT equivalent to REAL.
+					"double_col FLOAT(24)," +   // A precision of 24 or greater makes FLOAT equivalent to DOUBLE PRECISION.
+					"decimal_col DECIMAL(10, 4))");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"1, TIMESTAMP('2020-01-01 15:35:00.123456'), TIMESTAMP('2020-01-01 15:35:00.123456789'), " +
+					"TIME('15:35:00'), 1.175E-37, 1.79769E+308, 100.1234)");
+			statement.executeUpdate("INSERT INTO " + INPUT_TABLE + " VALUES (" +
+					"2, TIMESTAMP('2020-01-01 15:36:01.123456'), TIMESTAMP('2020-01-01 15:36:01.123456789'), " +
+					"TIME('15:36:01'), -1.175E-37, -1.79769E+308, 101.1234)");
+		}
+	}
+
+	@After
+	public void clearOutputTable() throws Exception {
+		Class.forName(DRIVER_CLASS);
+		try (
+			Connection conn = DriverManager.getConnection(DB_URL);
+			Statement stat = conn.createStatement()) {
+			stat.execute("DROP TABLE " + INPUT_TABLE);
+		}
 	}
 
 	@Test
-	public void testFieldsProjection() throws Exception {
-		StreamITCase.clear();
+	public void testJDBCSource() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
+			.useBlinkPlanner()
+			.inStreamingMode()
+			.build();
+		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
+
+		tEnv.sqlUpdate(
+			"CREATE TABLE " + INPUT_TABLE + "(" +
+				"id BIGINT," +
+				"timestamp6_col TIMESTAMP(6)," +
+				"timestamp9_col TIMESTAMP(9)," +
+				"time_col TIME," +
+				"real_col FLOAT," +
+				"double_col DOUBLE," +
+				"decimal_col DECIMAL(10, 4)" +
+				") WITH (" +
+				"  'connector.type'='jdbc'," +
+				"  'connector.url'='" + DB_URL + "'," +
+				"  'connector.table'='" + INPUT_TABLE + "'" +
+				")"
+		);
 
-		Table result = tEnv.sqlQuery(SELECT_ID_BOOKS);
-		DataStream<Row> resultSet = tEnv.toAppendStream(result, Row.class);
-		resultSet.addSink(new StreamITCase.StringSink<>());
+		StreamITCase.clear();
+		tEnv.toAppendStream(tEnv.sqlQuery("SELECT * FROM " + INPUT_TABLE), Row.class)
+			.addSink(new StreamITCase.StringSink<>());
 		env.execute();
 
-		List<String> expected = new ArrayList<>();
-		expected.add("1001");
-		expected.add("1002");
-		expected.add("1003");
-		expected.add("1004");
-		expected.add("1005");
-		expected.add("1006");
-		expected.add("1007");
-		expected.add("1008");
-		expected.add("1009");
-		expected.add("1010");
-
+		List<String> expected =
+			Arrays.asList(
+				"1,2020-01-01T15:35:00.123456,2020-01-01T15:35:00.123456789,15:35,1.175E-37,1.79769E308,100.1234",
+				"2,2020-01-01T15:36:01.123456,2020-01-01T15:36:01.123456789,15:36:01,-1.175E-37,-1.79769E308,101.1234");
 		StreamITCase.compareWithList(expected);
 	}
 
 	@Test
-	public void testAllFieldsSelection() throws Exception {
-		StreamITCase.clear();
+	public void testProjectableJDBCSource() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
+				.useBlinkPlanner()
+				.inStreamingMode()
+				.build();
+		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
+
+		tEnv.sqlUpdate(
+			"CREATE TABLE " + INPUT_TABLE + "(" +
+				"id BIGINT," +
+				"timestamp6_col TIMESTAMP(6)," +
+				"timestamp9_col TIMESTAMP(9)," +
+				"time_col TIME," +
+				"real_col FLOAT," +
+				"decimal_col DECIMAL(10, 4)" +
+				") WITH (" +
+				"  'connector.type'='jdbc'," +
+				"  'connector.url'='" + DB_URL + "'," +
+				"  'connector.table'='" + INPUT_TABLE + "'" +
+				")"
+		);
 
-		Table result = tEnv.sqlQuery(SELECT_ALL_BOOKS);
-		DataStream<Row> resultSet = tEnv.toAppendStream(result, Row.class);
-		resultSet.addSink(new StreamITCase.StringSink<>());
+		StreamITCase.clear();
+		tEnv.toAppendStream(tEnv.sqlQuery("SELECT timestamp6_col, decimal_col FROM " + INPUT_TABLE), Row.class)
+				.addSink(new StreamITCase.StringSink<>());
 		env.execute();
 
-		List<String> expected = new ArrayList<>();
-		expected.add("1001,Java public for dummies,Tan Ah Teck,11.11,11");
-		expected.add("1002,More Java for dummies,Tan Ah Teck,22.22,22");
-		expected.add("1003,More Java for more dummies,Mohammad Ali,33.33,33");
-		expected.add("1004,A Cup of Java,Kumar,44.44,44");
-		expected.add("1005,A Teaspoon of Java,Kevin Jones,55.55,55");
-		expected.add("1006,A Teaspoon of Java 1.4,Kevin Jones,66.66,66");
-		expected.add("1007,A Teaspoon of Java 1.5,Kevin Jones,77.77,77");
-		expected.add("1008,A Teaspoon of Java 1.6,Kevin Jones,88.88,88");
-		expected.add("1009,A Teaspoon of Java 1.7,Kevin Jones,99.99,99");
-		expected.add("1010,A Teaspoon of Java 1.8,Kevin Jones,null,1010");
-
+		List<String> expected =
+			Arrays.asList(
+				"2020-01-01T15:35:00.123456,100.1234",
+				"2020-01-01T15:36:01.123456,101.1234");
 		StreamITCase.compareWithList(expected);
 	}
 
+	@Test(expected = TableException.class)
+	public void testInvalidPrecisionOfJDBCSource() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
+				.useBlinkPlanner()
+				.inStreamingMode()
+				.build();
+		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
+
+		tEnv.sqlUpdate(
+		"CREATE TABLE " + INPUT_TABLE + "(" +
 
 Review comment:
   nice catch

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services