aljoscha commented on a change in pull request #11986:
URL: https://github.com/apache/flink/pull/11986#discussion_r421365481
##########
File path: docs/dev/table/connect.md
##########
@@ -1284,7 +1284,11 @@ CREATE TABLE MyUserTable (
'connector.username' = 'name',
'connector.password' = 'password',
- -- **followings are scan options, optional, used when reading from table**
+ -- **followings are scan options, optional, used when reading from a table**
+
+ -- optional: SQL query / prepared statement.
+ -- If set, it will takes over the 'connector.table' setting
+ 'connector.read.query' = 'SELECT * FROM sometable',
Review comment:
```suggestion
-- optional: SQL query / prepared statement.
-- If set, this will take precedence over the 'connector.table' setting
'connector.read.query' = 'SELECT * FROM sometable',
```
##########
File path:
flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceITCase.java
##########
@@ -153,4 +153,34 @@ public void testProjectableJDBCSource() throws Exception {
"2020-01-01T15:36:01.123456,101.1234");
StreamITCase.compareWithList(expected);
}
+
+ @Test
+ public void testScanQueryJDBCSource() throws Exception {
+ StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+ EnvironmentSettings envSettings =
EnvironmentSettings.newInstance()
+ .useBlinkPlanner()
+ .inStreamingMode()
+ .build();
+ StreamTableEnvironment tEnv =
StreamTableEnvironment.create(env, envSettings);
+
+ final String testQuery = "SELECT id FROM " + INPUT_TABLE;
+ tEnv.sqlUpdate(
+ "CREATE TABLE test(" +
+ "id BIGINT" +
+ ") WITH (" +
+ " 'connector.type'='jdbc'," +
+ " 'connector.url'='" + DB_URL + "'," +
+ " 'connector.table'='whatever'," +
+ " 'connector.read.query'='" + testQuery + "'" +
+ ")"
+ );
+
+ StreamITCase.clear();
Review comment:
I think using the static `StreamITCase` functionality here is an
anti-pattern that we should avoid. (I know that it's widely used in the code
base, but we should rid of it at some point.)
Ideally, with the FLIP-84 work it should be as easy as
```
TableResult tableResult = tEnv.executeSql("SELECT timestamp6_col,
decimal_col FROM " + INPUT_TABLE);
tableResult.collect();
```
but that is not yet in master. I'm inclined to wait for a but but basically
have this PR ready in a mergable state.
----------------------------------------------------------------
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:
[email protected]