Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-17361] [jdbc] Added custom query on JDBC tables #11986

Merged
merged 2 commits into from
May 19, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
6 changes: 5 additions & 1 deletion docs/dev/table/connect.md
Original file line number Diff line number Diff line change
Expand Up @@ -1307,7 +1307,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, this will take precedence over the 'connector.table' setting
'connector.read.query' = 'SELECT * FROM sometable',

-- These options must all be specified if any of them is specified. In addition,
-- partition.num must be specified. They describe how to partition the table when
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
*/
public class JdbcReadOptions implements Serializable {

private final String query;
private final String partitionColumnName;
private final Long partitionLowerBound;
private final Long partitionUpperBound;
Expand All @@ -35,11 +36,13 @@ public class JdbcReadOptions implements Serializable {
private final int fetchSize;

private JdbcReadOptions(
String query,
String partitionColumnName,
Long partitionLowerBound,
Long partitionUpperBound,
Integer numPartitions,
int fetchSize) {
this.query = query;
this.partitionColumnName = partitionColumnName;
this.partitionLowerBound = partitionLowerBound;
this.partitionUpperBound = partitionUpperBound;
Expand All @@ -48,6 +51,10 @@ private JdbcReadOptions(
this.fetchSize = fetchSize;
}

public Optional<String> getQuery() {
return Optional.ofNullable(query);
}

public Optional<String> getPartitionColumnName() {
return Optional.ofNullable(partitionColumnName);
}
Expand Down Expand Up @@ -76,11 +83,12 @@ public static Builder builder() {
public boolean equals(Object o) {
if (o instanceof JdbcReadOptions) {
JdbcReadOptions options = (JdbcReadOptions) o;
return Objects.equals(partitionColumnName, options.partitionColumnName) &&
Objects.equals(partitionLowerBound, options.partitionLowerBound) &&
Objects.equals(partitionUpperBound, options.partitionUpperBound) &&
Objects.equals(numPartitions, options.numPartitions) &&
Objects.equals(fetchSize, options.fetchSize);
return Objects.equals(query, options.query) &&
Objects.equals(partitionColumnName, options.partitionColumnName) &&
Objects.equals(partitionLowerBound, options.partitionLowerBound) &&
Objects.equals(partitionUpperBound, options.partitionUpperBound) &&
Objects.equals(numPartitions, options.numPartitions) &&
Objects.equals(fetchSize, options.fetchSize);
} else {
return false;
}
Expand All @@ -90,13 +98,22 @@ public boolean equals(Object o) {
* Builder of {@link JdbcReadOptions}.
*/
public static class Builder {
protected String query;
protected String partitionColumnName;
protected Long partitionLowerBound;
protected Long partitionUpperBound;
protected Integer numPartitions;

protected int fetchSize = 0;

/**
* optional, SQL query statement for this JDBC source.
*/
public Builder setQuery(String query) {
this.query = query;
return this;
}

/**
* optional, name of the column used for partitioning the input.
*/
Expand Down Expand Up @@ -140,7 +157,7 @@ public Builder setFetchSize(int fetchSize) {

public JdbcReadOptions build() {
return new JdbcReadOptions(
partitionColumnName, partitionLowerBound, partitionUpperBound, numPartitions, fetchSize);
query, partitionColumnName, partitionLowerBound, partitionUpperBound, numPartitions, fetchSize);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -168,8 +168,7 @@ private JdbcInputFormat getInputFormat() {
}

final JdbcDialect dialect = options.getDialect();
String query = dialect.getSelectFromStatement(
options.getTableName(), rowTypeInfo.getFieldNames(), new String[0]);
String query = getBaseQueryStatement(rowTypeInfo);
if (readOptions.getPartitionColumnName().isPresent()) {
long lowerBound = readOptions.getPartitionLowerBound().get();
long upperBound = readOptions.getPartitionUpperBound().get();
Expand All @@ -185,6 +184,12 @@ private JdbcInputFormat getInputFormat() {
return builder.finish();
}

private String getBaseQueryStatement(RowTypeInfo rowTypeInfo) {
return readOptions.getQuery().orElseGet(() ->
options.getDialect().getSelectFromStatement(
options.getTableName(), rowTypeInfo.getFieldNames(), new String[0]));
}

@Override
public boolean equals(Object o) {
if (o instanceof JdbcTableSource) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@
import static org.apache.flink.table.descriptors.JdbcValidator.CONNECTOR_READ_PARTITION_LOWER_BOUND;
import static org.apache.flink.table.descriptors.JdbcValidator.CONNECTOR_READ_PARTITION_NUM;
import static org.apache.flink.table.descriptors.JdbcValidator.CONNECTOR_READ_PARTITION_UPPER_BOUND;
import static org.apache.flink.table.descriptors.JdbcValidator.CONNECTOR_READ_QUERY;
import static org.apache.flink.table.descriptors.JdbcValidator.CONNECTOR_TABLE;
import static org.apache.flink.table.descriptors.JdbcValidator.CONNECTOR_TYPE_VALUE_JDBC;
import static org.apache.flink.table.descriptors.JdbcValidator.CONNECTOR_URL;
Expand Down Expand Up @@ -96,6 +97,7 @@ public List<String> supportedProperties() {
properties.add(CONNECTOR_PASSWORD);

// scan options
properties.add(CONNECTOR_READ_QUERY);
properties.add(CONNECTOR_READ_PARTITION_COLUMN);
properties.add(CONNECTOR_READ_PARTITION_NUM);
properties.add(CONNECTOR_READ_PARTITION_LOWER_BOUND);
Expand Down Expand Up @@ -184,13 +186,17 @@ private JdbcOptions getJdbcOptions(DescriptorProperties descriptorProperties) {
}

private JdbcReadOptions getJdbcReadOptions(DescriptorProperties descriptorProperties) {
final Optional<String> query = descriptorProperties.getOptionalString(CONNECTOR_READ_QUERY);
final Optional<String> partitionColumnName =
descriptorProperties.getOptionalString(CONNECTOR_READ_PARTITION_COLUMN);
final Optional<Long> partitionLower = descriptorProperties.getOptionalLong(CONNECTOR_READ_PARTITION_LOWER_BOUND);
final Optional<Long> partitionUpper = descriptorProperties.getOptionalLong(CONNECTOR_READ_PARTITION_UPPER_BOUND);
final Optional<Integer> numPartitions = descriptorProperties.getOptionalInt(CONNECTOR_READ_PARTITION_NUM);

final JdbcReadOptions.Builder builder = JdbcReadOptions.builder();
if (query.isPresent()) {
builder.setQuery(query.get());
}
if (partitionColumnName.isPresent()) {
builder.setPartitionColumnName(partitionColumnName.get());
builder.setPartitionLowerBound(partitionLower.get());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ public class JdbcValidator extends ConnectorDescriptorValidator {
public static final String CONNECTOR_USERNAME = "connector.username";
public static final String CONNECTOR_PASSWORD = "connector.password";

public static final String CONNECTOR_READ_QUERY = "connector.read.query";
public static final String CONNECTOR_READ_PARTITION_COLUMN = "connector.read.partition.column";
public static final String CONNECTOR_READ_PARTITION_LOWER_BOUND = "connector.read.partition.lower-bound";
public static final String CONNECTOR_READ_PARTITION_UPPER_BOUND = "connector.read.partition.upper-bound";
Expand Down Expand Up @@ -89,6 +90,7 @@ private void validateCommonProperties(DescriptorProperties properties) {
}

private void validateReadProperties(DescriptorProperties properties) {
properties.validateString(CONNECTOR_READ_QUERY, true);
properties.validateString(CONNECTOR_READ_PARTITION_COLUMN, true);
properties.validateLong(CONNECTOR_READ_PARTITION_LOWER_BOUND, true);
properties.validateLong(CONNECTOR_READ_PARTITION_UPPER_BOUND, true);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,8 @@
import org.apache.flink.connector.jdbc.JdbcTestBase;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.TableResult;
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;

Expand All @@ -34,8 +34,15 @@
import java.sql.DriverManager;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import java.util.Spliterator;
import java.util.Spliterators;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;

import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.junit.Assert.assertThat;


/**
Expand Down Expand Up @@ -107,20 +114,19 @@ public void testJdbcSource() throws Exception {
")"
);

StreamITCase.clear();
tEnv.toAppendStream(tEnv.sqlQuery("SELECT * FROM " + INPUT_TABLE), Row.class)
.addSink(new StreamITCase.StringSink<>());
env.execute();
TableResult tableResult = tEnv.executeSql("SELECT * FROM " + INPUT_TABLE);

List<String> results = manifestResults(tableResult);

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);
assertThat(
results,
containsInAnyOrder(
"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"));
}

@Test
public void testProjectableJdbcSource() throws Exception {
public void testProjectableJdbcSource() {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
EnvironmentSettings envSettings = EnvironmentSettings.newInstance()
.useBlinkPlanner()
Expand All @@ -143,15 +149,50 @@ public void testProjectableJdbcSource() throws Exception {
")"
);

StreamITCase.clear();
tEnv.toAppendStream(tEnv.sqlQuery("SELECT timestamp6_col, decimal_col FROM " + INPUT_TABLE), Row.class)
.addSink(new StreamITCase.StringSink<>());
env.execute();
TableResult tableResult = tEnv.executeSql("SELECT timestamp6_col, decimal_col FROM " + INPUT_TABLE);

List<String> results = manifestResults(tableResult);

assertThat(
results,
containsInAnyOrder(
"2020-01-01T15:35:00.123456,100.1234",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@aljoscha for the future: please use instances instead of strings

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Noted, that's a very good point.

"2020-01-01T15:36:01.123456,101.1234"));
}

@Test
public void testScanQueryJDBCSource() {
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.executeSql(
"CREATE TABLE test(" +
"id BIGINT" +
") WITH (" +
" 'connector.type'='jdbc'," +
" 'connector.url'='" + DB_URL + "'," +
" 'connector.table'='whatever'," +
" 'connector.read.query'='" + testQuery + "'" +
")"
);

TableResult tableResult = tEnv.executeSql("SELECT id FROM test");

List<String> results = manifestResults(tableResult);

assertThat(results, containsInAnyOrder("1", "2"));
}

List<String> expected =
Arrays.asList(
"2020-01-01T15:35:00.123456,100.1234",
"2020-01-01T15:36:01.123456,101.1234");
StreamITCase.compareWithList(expected);
private static List<String> manifestResults(TableResult result) {
Iterator<Row> resultIterator = result.collect();
return StreamSupport
.stream(Spliterators.spliteratorUnknownSize(resultIterator, Spliterator.ORDERED), false)
.map(Row::toString)
.collect(Collectors.toList());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,7 @@ public void testJdbcCommonProperties() {
@Test
public void testJdbcReadProperties() {
Map<String, String> properties = getBasicProperties();
properties.put("connector.read.query", "SELECT aaa FROM mytable");
properties.put("connector.read.partition.column", "aaa");
properties.put("connector.read.partition.lower-bound", "-10");
properties.put("connector.read.partition.upper-bound", "100");
Expand All @@ -102,6 +103,7 @@ public void testJdbcReadProperties() {
.setTableName("mytable")
.build();
final JdbcReadOptions readOptions = JdbcReadOptions.builder()
.setQuery("SELECT aaa FROM mytable")
.setPartitionColumnName("aaa")
.setPartitionLowerBound(-10)
.setPartitionUpperBound(100)
Expand Down