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-33463][Connector/JDBC] Support the implementation of dynamic source tables based on the new source #117

Merged
merged 2 commits into from
May 9, 2024
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.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,9 @@ protected JdbcConnectionOptions(
@Nullable String driverName,
int connectionCheckTimeoutSeconds,
@Nonnull Properties properties) {
Preconditions.checkArgument(connectionCheckTimeoutSeconds > 0);
Preconditions.checkArgument(
connectionCheckTimeoutSeconds > 0,
"Connection check timeout seconds shouldn't be smaller than 1");
this.url = Preconditions.checkNotNull(url, "jdbc url is empty");
this.driverName = driverName;
this.connectionCheckTimeoutSeconds = connectionCheckTimeoutSeconds;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -143,9 +143,6 @@ public JdbcSourceBuilder<OUT> setResultExtractor(ResultExtractor<OUT> resultExtr
}

public JdbcSourceBuilder<OUT> setUsername(String username) {
Preconditions.checkArgument(
!StringUtils.isNullOrWhitespaceOnly(username),
"It's required to set the 'username'.");
connOptionsBuilder.withUsername(username);
return this;
}
Expand Down Expand Up @@ -180,6 +177,12 @@ public JdbcSourceBuilder<OUT> setTypeInformation(

// ------ Optional ------------------------------------------------------------------

public JdbcSourceBuilder<OUT> setConnectionCheckTimeoutSeconds(
int connectionCheckTimeoutSeconds) {
connOptionsBuilder.withConnectionCheckTimeoutSeconds(connectionCheckTimeoutSeconds);
return this;
}

public JdbcSourceBuilder<OUT> setConnectionProperty(String propKey, String propVal) {
Preconditions.checkNotNull(propKey, "Connection property key mustn't be null");
Preconditions.checkNotNull(propVal, "Connection property value mustn't be null");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,17 +19,25 @@
package org.apache.flink.connector.jdbc.table;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.connector.source.Boundedness;
import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
import org.apache.flink.connector.jdbc.internal.options.InternalJdbcConnectionOptions;
import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
import org.apache.flink.connector.jdbc.source.JdbcSource;
import org.apache.flink.connector.jdbc.source.JdbcSourceBuilder;
import org.apache.flink.connector.jdbc.split.CompositeJdbcParameterValuesProvider;
import org.apache.flink.connector.jdbc.split.JdbcGenericParameterValuesProvider;
import org.apache.flink.connector.jdbc.split.JdbcNumericBetweenParametersProvider;
import org.apache.flink.connector.jdbc.split.JdbcParameterValuesProvider;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSource;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.connector.Projection;
import org.apache.flink.table.connector.ProviderContext;
import org.apache.flink.table.connector.source.DataStreamScanProvider;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.InputFormatProvider;
import org.apache.flink.table.connector.source.LookupTableSource;
import org.apache.flink.table.connector.source.ScanTableSource;
import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown;
Expand All @@ -38,6 +46,7 @@
import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
import org.apache.flink.table.connector.source.lookup.PartialCachingLookupProvider;
import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.expressions.CallExpression;
import org.apache.flink.table.expressions.ResolvedExpression;
import org.apache.flink.table.types.DataType;
Expand Down Expand Up @@ -68,6 +77,8 @@ public class JdbcDynamicTableSource
SupportsFilterPushDown {
private static final Logger LOG = LoggerFactory.getLogger(JdbcDynamicTableSource.class);

private static final String JDBC_TRANSFORMATION = "jdbc";

private final InternalJdbcConnectionOptions options;
private final JdbcReadOptions readOptions;
private final int lookupMaxRetryTimes;
Expand Down Expand Up @@ -121,17 +132,18 @@ public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) {
}

@Override
public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) {
final JdbcRowDataInputFormat.Builder builder =
JdbcRowDataInputFormat.builder()
.setDrivername(options.getDriverName())
public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {

final JdbcSourceBuilder<RowData> builder =
JdbcSource.<RowData>builder()
.setDriverName(options.getDriverName())
.setDBUrl(options.getDbURL())
.setUsername(options.getUsername().orElse(null))
.setPassword(options.getPassword().orElse(null))
.setAutoCommit(readOptions.getAutoCommit());

if (readOptions.getFetchSize() != 0) {
builder.setFetchSize(readOptions.getFetchSize());
builder.setResultSetFetchSize(readOptions.getFetchSize());
}
final JdbcDialect dialect = options.getDialect();
String query =
Expand All @@ -153,19 +165,19 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderCon
.ofBatchNum(numPartitions),
new JdbcGenericParameterValuesProvider(allPushdownParams));

builder.setParametersProvider(allParams);
builder.setJdbcParameterValuesProvider(allParams);

predicates.add(
dialect.quoteIdentifier(readOptions.getPartitionColumnName().get())
+ " BETWEEN ? AND ?");
} else {
builder.setParametersProvider(
builder.setJdbcParameterValuesProvider(
new JdbcGenericParameterValuesProvider(replicatePushdownParamsForN(1)));
}

predicates.addAll(this.resolvedPredicates);

if (predicates.size() > 0) {
if (!predicates.isEmpty()) {
String joinedConditions =
predicates.stream()
.map(pred -> String.format("(%s)", pred))
Expand All @@ -179,13 +191,16 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderCon

LOG.debug("Query generated for JDBC scan: " + query);

builder.setQuery(query);
builder.setSql(query);
final RowType rowType = (RowType) physicalRowDataType.getLogicalType();
builder.setRowConverter(dialect.getRowConverter(rowType));
builder.setRowDataTypeInfo(
runtimeProviderContext.createTypeInformation(physicalRowDataType));

return InputFormatProvider.of(builder.build());
builder.setResultExtractor(new RowDataResultExtractor(dialect.getRowConverter(rowType)));
builder.setTypeInformation(scanContext.createTypeInformation(physicalRowDataType));
options.getProperties()
.forEach(
(key, value) ->
builder.setConnectionProperty(key.toString(), value.toString()));
JdbcSource<RowData> source = builder.build();
return new JdbcDataStreamScanProvider(source);
}

@Override
Expand Down Expand Up @@ -295,4 +310,30 @@ private Serializable[][] replicatePushdownParamsForN(int n) {
}
return allPushdownParams;
}

private static class JdbcDataStreamScanProvider implements DataStreamScanProvider {

private final JdbcSource<RowData> source;

public JdbcDataStreamScanProvider(JdbcSource<RowData> source) {
this.source = Preconditions.checkNotNull(source);
}

@Override
public DataStream<RowData> produceDataStream(
ProviderContext providerContext, StreamExecutionEnvironment execEnv) {
DataStreamSource<RowData> sourceStream =
execEnv.fromSource(
source,
WatermarkStrategy.noWatermarks(),
JdbcDynamicTableSource.class.getSimpleName());
providerContext.generateUid(JDBC_TRANSFORMATION).ifPresent(sourceStream::uid);
return sourceStream;
}

@Override
public boolean isBounded() {
return source.getBoundedness() == Boundedness.BOUNDED;
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.connector.jdbc.table;

import org.apache.flink.connector.jdbc.converter.JdbcRowConverter;
import org.apache.flink.connector.jdbc.source.reader.extractor.ResultExtractor;
import org.apache.flink.table.data.RowData;
import org.apache.flink.util.Preconditions;

import java.sql.ResultSet;
import java.sql.SQLException;

/** The result extractor for {@link RowData}. */
public class RowDataResultExtractor implements ResultExtractor<RowData> {

private final JdbcRowConverter jdbcRowConverter;

public RowDataResultExtractor(JdbcRowConverter jdbcRowConverter) {
this.jdbcRowConverter = Preconditions.checkNotNull(jdbcRowConverter);
}

@Override
public RowData extract(ResultSet resultSet) throws SQLException {
return jdbcRowConverter.toInternal(resultSet);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -116,8 +116,6 @@ void testSetResultSetFetchSize() {
void testSetConnectionInfo() {
assertThatThrownBy(() -> JdbcSource.builder().setDriverName(""))
.isInstanceOf(IllegalArgumentException.class);
assertThatThrownBy(() -> JdbcSource.builder().setUsername(""))
.isInstanceOf(IllegalArgumentException.class);
assertThatThrownBy(() -> JdbcSource.builder().setDBUrl(""))
.isInstanceOf(IllegalArgumentException.class);
}
Expand Down