org.apache.beam.sdk.io.jdbc.JdbcIO Maven / Gradle / Ivy
/*
* 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.beam.sdk.io.jdbc;
import static org.apache.beam.sdk.io.jdbc.SchemaUtil.checkNullabilityForFields;
import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull;
import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull;
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument;
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull;
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState;
import com.google.auto.value.AutoValue;
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
import java.io.IOException;
import java.io.Serializable;
import java.net.URLClassLoader;
import java.sql.Connection;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
import java.sql.ResultSetMetaData;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import javax.sql.DataSource;
import org.apache.beam.sdk.coders.CannotProvideCoderException;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.CoderRegistry;
import org.apache.beam.sdk.coders.RowCoder;
import org.apache.beam.sdk.coders.VoidCoder;
import org.apache.beam.sdk.io.jdbc.JdbcIO.WriteFn.WriteFnSpec;
import org.apache.beam.sdk.io.jdbc.JdbcUtil.PartitioningFn;
import org.apache.beam.sdk.io.jdbc.SchemaUtil.FieldWithIndex;
import org.apache.beam.sdk.metrics.Distribution;
import org.apache.beam.sdk.metrics.Metrics;
import org.apache.beam.sdk.options.ValueProvider;
import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
import org.apache.beam.sdk.schemas.NoSuchSchemaException;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.schemas.SchemaRegistry;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.Filter;
import org.apache.beam.sdk.transforms.GroupIntoBatches;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.Reshuffle;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.transforms.SerializableFunctions;
import org.apache.beam.sdk.transforms.SimpleFunction;
import org.apache.beam.sdk.transforms.Values;
import org.apache.beam.sdk.transforms.View;
import org.apache.beam.sdk.transforms.Wait;
import org.apache.beam.sdk.transforms.WithKeys;
import org.apache.beam.sdk.transforms.display.DisplayData;
import org.apache.beam.sdk.transforms.display.HasDisplayData;
import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
import org.apache.beam.sdk.util.BackOff;
import org.apache.beam.sdk.util.BackOffUtils;
import org.apache.beam.sdk.util.FluentBackoff;
import org.apache.beam.sdk.util.Sleeper;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollection.IsBounded;
import org.apache.beam.sdk.values.PCollectionView;
import org.apache.beam.sdk.values.PDone;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.sdk.values.TypeDescriptor;
import org.apache.beam.sdk.values.TypeDescriptors;
import org.apache.beam.sdk.values.TypeDescriptors.TypeVariableExtractor;
import org.apache.commons.dbcp2.BasicDataSource;
import org.apache.commons.dbcp2.DataSourceConnectionFactory;
import org.apache.commons.dbcp2.PoolableConnectionFactory;
import org.apache.commons.dbcp2.PoolingDataSource;
import org.apache.commons.pool2.impl.GenericObjectPool;
import org.apache.commons.pool2.impl.GenericObjectPoolConfig;
import org.checkerframework.checker.nullness.qual.NonNull;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.checkerframework.dataflow.qual.Pure;
import org.joda.time.Duration;
import org.joda.time.Instant;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* IO to read and write data on JDBC.
*
* Reading from JDBC datasource
*
* JdbcIO source returns a bounded collection of {@code T} as a {@code PCollection}. T is the
* type returned by the provided {@link RowMapper}.
*
* To configure the JDBC source, you have to provide a {@link DataSourceConfiguration} using
* 1. {@link DataSourceConfiguration#create(DataSource)}(which must be {@link Serializable});
* 2. or {@link DataSourceConfiguration#create(String, String)}(driver class name and url).
* Optionally, {@link DataSourceConfiguration#withUsername(String)} and {@link
* DataSourceConfiguration#withPassword(String)} allows you to define username and password.
*
*
For example:
*
*
{@code
* pipeline.apply(JdbcIO.>read()
* .withDataSourceConfiguration(JdbcIO.DataSourceConfiguration.create(
* "com.mysql.jdbc.Driver", "jdbc:mysql://hostname:3306/mydb")
* .withUsername("username")
* .withPassword("password"))
* .withQuery("select id,name from Person")
* .withRowMapper(new JdbcIO.RowMapper>() {
* public KV mapRow(ResultSet resultSet) throws Exception {
* return KV.of(resultSet.getInt(1), resultSet.getString(2));
* }
* })
* );
* }
*
* Note you should check with your database provider for the JDBC Driver and Connection Url that
* used to create the DataSourceConfiguration. For example, if you use Cloud SQL with postgres, the
* JDBC connection Url has this pattern with SocketFactory:
* "jdbc:postgresql://google/mydb?cloudSqlInstance=project:region:myinstance&
* socketFactory=com.google.cloud.sql.postgres.SocketFactory". Check
* here for more details.
*
*
Query parameters can be configured using a user-provided {@link StatementPreparator}. For
* example:
*
*
{@code
* pipeline.apply(JdbcIO.>read()
* .withDataSourceConfiguration(JdbcIO.DataSourceConfiguration.create(
* "com.mysql.jdbc.Driver", "jdbc:mysql://hostname:3306/mydb",
* "username", "password"))
* .withQuery("select id,name from Person where name = ?")
* .withStatementPreparator(new JdbcIO.StatementPreparator() {
* public void setParameters(PreparedStatement preparedStatement) throws Exception {
* preparedStatement.setString(1, "Darwin");
* }
* })
* .withRowMapper(new JdbcIO.RowMapper>() {
* public KV mapRow(ResultSet resultSet) throws Exception {
* return KV.of(resultSet.getInt(1), resultSet.getString(2));
* }
* })
* );
* }
*
* To customize the building of the {@link DataSource} we can provide a {@link
* SerializableFunction}. For example if you need to provide a {@link PoolingDataSource} from an
* existing {@link DataSourceConfiguration}: you can use a {@link PoolableDataSourceProvider}:
*
*
{@code
* pipeline.apply(JdbcIO.>read()
* .withDataSourceProviderFn(JdbcIO.PoolableDataSourceProvider.of(
* JdbcIO.DataSourceConfiguration.create(
* "com.mysql.jdbc.Driver", "jdbc:mysql://hostname:3306/mydb",
* "username", "password")))
* // ...
* );
* }
*
* By default, the provided function requests a DataSource per execution thread. In some
* circumstances this can quickly overwhelm the database by requesting too many connections. In that
* case you should look into sharing a single instance of a {@link PoolingDataSource} across all the
* execution threads. For example:
*
*
* private static class MyDataSourceProviderFn implements{@literal SerializableFunction} {
* private static transient DataSource dataSource;
*
* {@literal @Override}
* public synchronized DataSource apply(Void input) {
* if (dataSource == null) {
* dataSource = ... build data source ...
* }
* return dataSource;
* }
* }
* {@literal
* pipeline.apply(JdbcIO.>read()
* .withDataSourceProviderFn(new MyDataSourceProviderFn())
* // ...
* );
* }
*
* Parallel reading from a JDBC datasource
*
* Beam supports partitioned reading of all data from a table. Automatic partitioning is
* supported for a few data types: {@link Long}, {@link org.joda.time.DateTime}. To enable this, use
* {@link JdbcIO#readWithPartitions(TypeDescriptor)}. For other types, use {@link
* ReadWithPartitions#readWithPartitions(JdbcReadWithPartitionsHelper)} with custom {@link
* JdbcReadWithPartitionsHelper}.
*
*
The partitioning scheme depends on these parameters, which can be user-provided, or
* automatically inferred by Beam (for the supported types):
*
*
* - Upper bound
*
- Lower bound
*
- Number of partitions - when auto-inferred, the number of partitions defaults to the square
* root of the number of rows divided by 5 (i.e.: {@code Math.floor(Math.sqrt(numRows) / 5)}).
*
*
* To trigger auto-inference of these parameters, the user just needs to not provide them. To
* infer them automatically, Beam runs either of these statements:
*
*
* - {@code SELECT min(column), max(column), COUNT(*) from table} when none of the parameters is
* passed to the transform.
*
- {@code SELECT min(column), max(column) from table} when only number of partitions is
* provided, but not upper or lower bounds.
*
*
* Should I use this transform? Consider using this transform in the following situations:
*
*
* - The partitioning column is indexed. This will help speed up the range queries
*
- Use auto-inference if the queries for bound and partition inference are efficient to
* execute in your DBMS.
*
- The distribution of data over the partitioning column is roughly uniform. Uniformity
* is not mandatory, but this transform will work best in that situation.
*
*
* The following example shows usage of auto-inferred ranges, number of partitions, and
* schema
*
*
{@code
* pipeline.apply(JdbcIO.readWithPartitions()
* .withDataSourceConfiguration(JdbcIO.DataSourceConfiguration.create(
* "com.mysql.jdbc.Driver", "jdbc:mysql://hostname:3306/mydb")
* .withUsername("username")
* .withPassword("password"))
* .withTable("Person")
* .withPartitionColumn("id")
* .withRowOutput()
* );
* }
*
* Instead of a full table you could also use a subquery in parentheses. The subquery can be
* specified using Table option instead and partition columns can be qualified using the subquery
* alias provided as part of Table. Note that a subquery may not perform as well with
* auto-inferred ranges and partitions, because it may not rely on indices to speed up the
* partitioning.
*
*
{@code
* pipeline.apply(JdbcIO.>readWithPartitions()
* .withDataSourceConfiguration(JdbcIO.DataSourceConfiguration.create(
* "com.mysql.jdbc.Driver", "jdbc:mysql://hostname:3306/mydb")
* .withUsername("username")
* .withPassword("password"))
* .withTable("(select id, name from Person) as subq")
* .withPartitionColumn("id")
* .withLowerBound(0)
* .withUpperBound(1000)
* .withNumPartitions(5)
* .withRowMapper(new JdbcIO.RowMapper>() {
* public KV mapRow(ResultSet resultSet) throws Exception {
* return KV.of(resultSet.getInt(1), resultSet.getString(2));
* }
* })
* );
* }
*
* Writing to JDBC datasource
*
* JDBC sink supports writing records into a database. It writes a {@link PCollection} to the
* database by converting each T into a {@link PreparedStatement} via a user-provided {@link
* PreparedStatementSetter}.
*
*
Like the source, to configure the sink, you have to provide a {@link DataSourceConfiguration}.
*
*
{@code
* pipeline
* .apply(...)
* .apply(JdbcIO.>write()
* .withDataSourceConfiguration(JdbcIO.DataSourceConfiguration.create(
* "com.mysql.jdbc.Driver", "jdbc:mysql://hostname:3306/mydb")
* .withUsername("username")
* .withPassword("password"))
* .withStatement("insert into Person values(?, ?)")
* .withPreparedStatementSetter(new JdbcIO.PreparedStatementSetter>() {
* public void setParameters(KV element, PreparedStatement query)
* throws SQLException {
* query.setInt(1, element.getKey());
* query.setString(2, element.getValue());
* }
* })
* );
* }
*
* NB: in case of transient failures, Beam runners may execute parts of JdbcIO.Write multiple
* times for fault tolerance. Because of that, you should avoid using {@code INSERT} statements,
* since that risks duplicating records in the database, or failing due to primary key conflicts.
* Consider using MERGE ("upsert")
* statements supported by your database instead.
*/
@SuppressWarnings({
"rawtypes" // TODO(https://github.com/apache/beam/issues/20447)
})
public class JdbcIO {
private static final Logger LOG = LoggerFactory.getLogger(JdbcIO.class);
/**
* Read data from a JDBC datasource.
*
* @param Type of the data to be read.
*/
public static Read read() {
return new AutoValue_JdbcIO_Read.Builder()
.setFetchSize(DEFAULT_FETCH_SIZE)
.setOutputParallelization(true)
.build();
}
/** Read Beam {@link Row}s from a JDBC data source. */
public static ReadRows readRows() {
return new AutoValue_JdbcIO_ReadRows.Builder()
.setFetchSize(DEFAULT_FETCH_SIZE)
.setOutputParallelization(true)
.setStatementPreparator(ignored -> {})
.build();
}
/**
* Like {@link #read}, but executes multiple instances of the query substituting each element of a
* {@link PCollection} as query parameters.
*
* @param Type of the data representing query parameters.
* @param Type of the data to be read.
*/
public static ReadAll readAll() {
return new AutoValue_JdbcIO_ReadAll.Builder()
.setFetchSize(DEFAULT_FETCH_SIZE)
.setOutputParallelization(true)
.build();
}
/**
* Like {@link #readAll}, but executes multiple instances of the query on the same table
* (subquery) using ranges.
*
* @param partitioningColumnType Type descriptor for the partition column.
* @param Type of the data to be read.
*/
public static ReadWithPartitions readWithPartitions(
TypeDescriptor partitioningColumnType) {
return new AutoValue_JdbcIO_ReadWithPartitions.Builder()
.setPartitionColumnType(partitioningColumnType)
.setNumPartitions(DEFAULT_NUM_PARTITIONS)
.setFetchSize(DEFAULT_FETCH_SIZE)
.setUseBeamSchema(false)
.build();
}
/**
* Like {@link #readAll}, but executes multiple instances of the query on the same table
* (subquery) using ranges.
*
* @param partitionsHelper Custom helper for defining partitions.
* @param Type of the data to be read.
*/
public static ReadWithPartitions readWithPartitions(
JdbcReadWithPartitionsHelper partitionsHelper) {
return new AutoValue_JdbcIO_ReadWithPartitions.Builder()
.setPartitionsHelper(partitionsHelper)
.setNumPartitions(DEFAULT_NUM_PARTITIONS)
.setFetchSize(DEFAULT_FETCH_SIZE)
.setUseBeamSchema(false)
.build();
}
public static ReadWithPartitions readWithPartitions() {
return JdbcIO.readWithPartitions(TypeDescriptors.longs());
}
private static final long DEFAULT_BATCH_SIZE = 1000L;
private static final long DEFAULT_MAX_BATCH_BUFFERING_DURATION = 200L;
private static final int DEFAULT_FETCH_SIZE = 50_000;
// Default values used from fluent backoff.
private static final Duration DEFAULT_INITIAL_BACKOFF = Duration.standardSeconds(1);
private static final Duration DEFAULT_MAX_CUMULATIVE_BACKOFF = Duration.standardDays(1000);
// Default value used for partitioning a table
private static final int DEFAULT_NUM_PARTITIONS = 200;
/**
* Write data to a JDBC datasource.
*
* @param Type of the data to be written.
*/
public static Write write() {
return new Write<>();
}
public static WriteVoid writeVoid() {
return new AutoValue_JdbcIO_WriteVoid.Builder()
.setBatchSize(DEFAULT_BATCH_SIZE)
.setMaxBatchBufferingDuration(DEFAULT_MAX_BATCH_BUFFERING_DURATION)
.setRetryStrategy(new DefaultRetryStrategy())
.setRetryConfiguration(RetryConfiguration.create(5, null, Duration.standardSeconds(5)))
.build();
}
/**
* This is the default {@link Predicate} we use to detect DeadLock. It basically test if the
* {@link SQLException#getSQLState()} equals 40001 or 40P01. 40001 is the SQL State used by most
* of databases to identify deadlock, and 40P01 is specific to PostgreSQL (see PostgreSQL documentation).
*/
public static class DefaultRetryStrategy implements RetryStrategy {
private static final Set errorCodesToRetry =
new HashSet(Arrays.asList("40001", "40P01"));
@Override
public boolean apply(SQLException e) {
String sqlState = e.getSQLState();
return sqlState != null && errorCodesToRetry.contains(sqlState);
}
}
private JdbcIO() {}
/**
* An interface used by {@link JdbcIO.Read} for converting each row of the {@link ResultSet} into
* an element of the resulting {@link PCollection}.
*/
@FunctionalInterface
public interface RowMapper extends Serializable {
T mapRow(ResultSet resultSet) throws Exception;
}
/**
* A POJO describing a {@link DataSource}, either providing directly a {@link DataSource} or all
* properties allowing to create a {@link DataSource}.
*/
@AutoValue
public abstract static class DataSourceConfiguration implements Serializable {
@Pure
abstract @Nullable ValueProvider getDriverClassName();
@Pure
abstract @Nullable ValueProvider getUrl();
@Pure
abstract @Nullable ValueProvider<@Nullable String> getUsername();
@Pure
abstract @Nullable ValueProvider<@Nullable String> getPassword();
@Pure
abstract @Nullable ValueProvider getConnectionProperties();
@Pure
abstract @Nullable ValueProvider> getConnectionInitSqls();
@Pure
abstract @Nullable ValueProvider getMaxConnections();
@Pure
abstract @Nullable ClassLoader getDriverClassLoader();
@Pure
abstract @Nullable ValueProvider getDriverJars();
@Pure
abstract @Nullable DataSource getDataSource();
abstract Builder builder();
@AutoValue.Builder
abstract static class Builder {
abstract Builder setDriverClassName(ValueProvider<@Nullable String> driverClassName);
abstract Builder setUrl(ValueProvider<@Nullable String> url);
abstract Builder setUsername(ValueProvider<@Nullable String> username);
abstract Builder setPassword(ValueProvider<@Nullable String> password);
abstract Builder setConnectionProperties(
ValueProvider<@Nullable String> connectionProperties);
abstract Builder setConnectionInitSqls(
ValueProvider> connectionInitSqls);
abstract Builder setMaxConnections(ValueProvider<@Nullable Integer> maxConnections);
abstract Builder setDriverClassLoader(ClassLoader driverClassLoader);
abstract Builder setDriverJars(ValueProvider driverJars);
abstract Builder setDataSource(@Nullable DataSource dataSource);
abstract DataSourceConfiguration build();
}
public static DataSourceConfiguration create(DataSource dataSource) {
checkArgument(dataSource != null, "dataSource can not be null");
checkArgument(dataSource instanceof Serializable, "dataSource must be Serializable");
return new AutoValue_JdbcIO_DataSourceConfiguration.Builder()
.setDataSource(dataSource)
.build();
}
public static DataSourceConfiguration create(String driverClassName, String url) {
checkArgument(driverClassName != null, "driverClassName can not be null");
checkArgument(url != null, "url can not be null");
return create(
ValueProvider.StaticValueProvider.of(driverClassName),
ValueProvider.StaticValueProvider.of(url));
}
public static DataSourceConfiguration create(
ValueProvider<@Nullable String> driverClassName, ValueProvider<@Nullable String> url) {
checkArgument(driverClassName != null, "driverClassName can not be null");
checkArgument(url != null, "url can not be null");
return new AutoValue_JdbcIO_DataSourceConfiguration.Builder()
.setDriverClassName(driverClassName)
.setUrl(url)
.build();
}
public DataSourceConfiguration withUsername(@Nullable String username) {
return withUsername(ValueProvider.StaticValueProvider.of(username));
}
public DataSourceConfiguration withUsername(ValueProvider<@Nullable String> username) {
return builder().setUsername(username).build();
}
public DataSourceConfiguration withPassword(@Nullable String password) {
return withPassword(ValueProvider.StaticValueProvider.of(password));
}
public DataSourceConfiguration withPassword(ValueProvider<@Nullable String> password) {
return builder().setPassword(password).build();
}
/**
* Sets the connection properties passed to driver.connect(...). Format of the string must be
* [propertyName=property;]*
*
* NOTE - The "user" and "password" properties can be add via {@link #withUsername(String)},
* {@link #withPassword(String)}, so they do not need to be included here.
*/
public DataSourceConfiguration withConnectionProperties(String connectionProperties) {
checkArgument(connectionProperties != null, "connectionProperties can not be null");
return withConnectionProperties(ValueProvider.StaticValueProvider.of(connectionProperties));
}
/** Same as {@link #withConnectionProperties(String)} but accepting a ValueProvider. */
public DataSourceConfiguration withConnectionProperties(
ValueProvider<@Nullable String> connectionProperties) {
checkArgument(connectionProperties != null, "connectionProperties can not be null");
return builder().setConnectionProperties(connectionProperties).build();
}
/**
* Sets the connection init sql statements to driver.connect(...).
*
*
NOTE - This property is not applicable across databases. Only MySQL and MariaDB support
* this. A Sql exception is thrown if your database does not support it.
*/
public DataSourceConfiguration withConnectionInitSqls(
Collection<@Nullable String> connectionInitSqls) {
checkArgument(connectionInitSqls != null, "connectionInitSqls can not be null");
return withConnectionInitSqls(ValueProvider.StaticValueProvider.of(connectionInitSqls));
}
/** Same as {@link #withConnectionInitSqls(Collection)} but accepting a ValueProvider. */
public DataSourceConfiguration withConnectionInitSqls(
ValueProvider> connectionInitSqls) {
checkArgument(connectionInitSqls != null, "connectionInitSqls can not be null");
checkArgument(!connectionInitSqls.get().isEmpty(), "connectionInitSqls can not be empty");
return builder().setConnectionInitSqls(connectionInitSqls).build();
}
/** Sets the maximum total number of connections. Use a negative value for no limit. */
public DataSourceConfiguration withMaxConnections(Integer maxConnections) {
checkArgument(maxConnections != null, "maxConnections can not be null");
return withMaxConnections(ValueProvider.StaticValueProvider.of(maxConnections));
}
/** Same as {@link #withMaxConnections(Integer)} but accepting a ValueProvider. */
public DataSourceConfiguration withMaxConnections(
ValueProvider<@Nullable Integer> maxConnections) {
return builder().setMaxConnections(maxConnections).build();
}
/**
* Sets the class loader instance to be used to load the JDBC driver. If not specified, the
* default class loader is used.
*/
public DataSourceConfiguration withDriverClassLoader(ClassLoader driverClassLoader) {
checkArgument(driverClassLoader != null, "driverClassLoader can not be null");
return builder().setDriverClassLoader(driverClassLoader).build();
}
/**
* Comma separated paths for JDBC drivers. This method is filesystem agnostic and can be used
* for all FileSystems supported by Beam If not specified, the default classloader is used to
* load the jars.
*
* For example, gs://your-bucket/driver_jar1.jar,gs://your-bucket/driver_jar2.jar.
*/
public DataSourceConfiguration withDriverJars(String driverJars) {
checkArgument(driverJars != null, "driverJars can not be null");
return withDriverJars(ValueProvider.StaticValueProvider.of(driverJars));
}
/** Same as {@link #withDriverJars(String)} but accepting a ValueProvider. */
public DataSourceConfiguration withDriverJars(ValueProvider driverJars) {
checkArgument(driverJars != null, "driverJars can not be null");
return builder().setDriverJars(driverJars).build();
}
void populateDisplayData(DisplayData.Builder builder) {
if (getDataSource() != null) {
builder.addIfNotNull(DisplayData.item("dataSource", getDataSource().getClass().getName()));
} else {
builder.addIfNotNull(DisplayData.item("jdbcDriverClassName", getDriverClassName()));
builder.addIfNotNull(DisplayData.item("jdbcUrl", getUrl()));
builder.addIfNotNull(DisplayData.item("username", getUsername()));
builder.addIfNotNull(DisplayData.item("driverJars", getDriverJars()));
}
}
public DataSource buildDatasource() {
if (getDataSource() == null) {
BasicDataSource basicDataSource = new BasicDataSource();
if (getDriverClassName() != null) {
basicDataSource.setDriverClassName(getDriverClassName().get());
}
if (getUrl() != null) {
basicDataSource.setUrl(getUrl().get());
}
if (getUsername() != null) {
@SuppressWarnings(
"nullness") // this is actually nullable, but apache commons dbcp2 not annotated
@NonNull
String username = getUsername().get();
basicDataSource.setUsername(username);
}
if (getPassword() != null) {
@SuppressWarnings(
"nullness") // this is actually nullable, but apache commons dbcp2 not annotated
@NonNull
String password = getPassword().get();
basicDataSource.setPassword(password);
}
if (getConnectionProperties() != null) {
String connectionProperties = getConnectionProperties().get();
if (connectionProperties != null) {
basicDataSource.setConnectionProperties(connectionProperties);
}
}
if (getConnectionInitSqls() != null) {
Collection connectionInitSqls = getConnectionInitSqls().get();
if (connectionInitSqls != null && !connectionInitSqls.isEmpty()) {
basicDataSource.setConnectionInitSqls(connectionInitSqls);
}
}
if (getMaxConnections() != null) {
Integer maxConnections = getMaxConnections().get();
if (maxConnections != null) {
basicDataSource.setMaxTotal(maxConnections);
}
}
if (getDriverClassLoader() != null) {
basicDataSource.setDriverClassLoader(getDriverClassLoader());
}
if (getDriverJars() != null) {
URLClassLoader classLoader =
URLClassLoader.newInstance(JdbcUtil.saveFilesLocally(getDriverJars().get()));
basicDataSource.setDriverClassLoader(classLoader);
}
return basicDataSource;
}
return getDataSource();
}
}
/**
* An interface used by the JdbcIO Write to set the parameters of the {@link PreparedStatement}
* used to setParameters into the database.
*/
@FunctionalInterface
public interface StatementPreparator extends Serializable {
void setParameters(PreparedStatement preparedStatement) throws Exception;
}
/** Implementation of {@link #readRows()}. */
@AutoValue
public abstract static class ReadRows extends PTransform> {
@Pure
abstract @Nullable SerializableFunction getDataSourceProviderFn();
@Pure
abstract @Nullable ValueProvider getQuery();
@Pure
abstract @Nullable StatementPreparator getStatementPreparator();
@Pure
abstract int getFetchSize();
@Pure
abstract boolean getOutputParallelization();
abstract Builder toBuilder();
@AutoValue.Builder
abstract static class Builder {
abstract Builder setDataSourceProviderFn(
SerializableFunction dataSourceProviderFn);
abstract Builder setQuery(ValueProvider query);
abstract Builder setStatementPreparator(StatementPreparator statementPreparator);
abstract Builder setFetchSize(int fetchSize);
abstract Builder setOutputParallelization(boolean outputParallelization);
abstract ReadRows build();
}
public ReadRows withDataSourceConfiguration(DataSourceConfiguration config) {
return withDataSourceProviderFn(new DataSourceProviderFromDataSourceConfiguration(config));
}
public ReadRows withDataSourceProviderFn(
SerializableFunction dataSourceProviderFn) {
return toBuilder().setDataSourceProviderFn(dataSourceProviderFn).build();
}
public ReadRows withQuery(String query) {
checkArgument(query != null, "query can not be null");
return withQuery(ValueProvider.StaticValueProvider.of(query));
}
public ReadRows withQuery(ValueProvider query) {
checkArgument(query != null, "query can not be null");
return toBuilder().setQuery(query).build();
}
public ReadRows withStatementPreparator(StatementPreparator statementPreparator) {
checkArgument(statementPreparator != null, "statementPreparator can not be null");
return toBuilder().setStatementPreparator(statementPreparator).build();
}
/**
* This method is used to set the size of the data that is going to be fetched and loaded in
* memory per every database call. Please refer to: {@link java.sql.Statement#setFetchSize(int)}
* It should ONLY be used if the default value throws memory errors.
*/
public ReadRows withFetchSize(int fetchSize) {
// Note that api.java.sql.Statement#setFetchSize says it only accepts values >= 0
// and that MySQL supports using Integer.MIN_VALUE as a hint to stream the ResultSet instead
// of loading it into memory. See
// https://dev.mysql.com/doc/connector-j/8.0/en/connector-j-reference-implementation-notes.html for additional details.
checkArgument(
fetchSize >= 0 || fetchSize == Integer.MIN_VALUE,
"fetch size must be >= 0 or equal to Integer.MIN_VALUE");
return toBuilder().setFetchSize(fetchSize).build();
}
/**
* Whether to reshuffle the resulting PCollection so results are distributed to all workers. The
* default is to parallelize and should only be changed if this is known to be unnecessary.
*/
public ReadRows withOutputParallelization(boolean outputParallelization) {
return toBuilder().setOutputParallelization(outputParallelization).build();
}
@Override
public PCollection expand(PBegin input) {
ValueProvider query = checkStateNotNull(getQuery(), "withQuery() is required");
SerializableFunction dataSourceProviderFn =
checkStateNotNull(
getDataSourceProviderFn(),
"withDataSourceConfiguration() or withDataSourceProviderFn() is required");
Schema schema = inferBeamSchema(dataSourceProviderFn.apply(null), query.get());
PCollection rows =
input.apply(
JdbcIO.read()
.withDataSourceProviderFn(dataSourceProviderFn)
.withQuery(query)
.withCoder(RowCoder.of(schema))
.withRowMapper(SchemaUtil.BeamRowMapper.of(schema))
.withFetchSize(getFetchSize())
.withOutputParallelization(getOutputParallelization())
.withStatementPreparator(checkStateNotNull(getStatementPreparator())));
rows.setRowSchema(schema);
return rows;
}
// Spotbugs seems to not understand the multi-statement try-with-resources
@SuppressFBWarnings("OBL_UNSATISFIED_OBLIGATION")
public static Schema inferBeamSchema(DataSource ds, String query) {
try (Connection conn = ds.getConnection();
PreparedStatement statement =
conn.prepareStatement(
query, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY)) {
ResultSetMetaData metadata =
checkStateNotNull(statement.getMetaData(), "could not get statement metadata");
return SchemaUtil.toBeamSchema(metadata);
} catch (SQLException e) {
throw new BeamSchemaInferenceException("Failed to infer Beam schema", e);
}
}
@Override
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder.add(DisplayData.item("query", getQuery()));
if (getDataSourceProviderFn() instanceof HasDisplayData) {
((HasDisplayData) getDataSourceProviderFn()).populateDisplayData(builder);
}
}
}
/** Implementation of {@link #read}. */
@AutoValue
public abstract static class Read extends PTransform> {
@Pure
abstract @Nullable SerializableFunction getDataSourceProviderFn();
@Pure
abstract @Nullable ValueProvider getQuery();
@Pure
abstract @Nullable StatementPreparator getStatementPreparator();
@Pure
abstract @Nullable RowMapper getRowMapper();
@Pure
abstract @Nullable Coder getCoder();
@Pure
abstract int getFetchSize();
@Pure
abstract boolean getOutputParallelization();
@Pure
abstract Builder toBuilder();
@AutoValue.Builder
abstract static class Builder {
abstract Builder setDataSourceProviderFn(
SerializableFunction dataSourceProviderFn);
abstract Builder setQuery(ValueProvider query);
abstract Builder setStatementPreparator(StatementPreparator statementPreparator);
abstract Builder setRowMapper(RowMapper rowMapper);
abstract Builder setCoder(Coder coder);
abstract Builder setFetchSize(int fetchSize);
abstract Builder setOutputParallelization(boolean outputParallelization);
abstract Read build();
}
public Read withDataSourceConfiguration(final DataSourceConfiguration config) {
return withDataSourceProviderFn(new DataSourceProviderFromDataSourceConfiguration(config));
}
public Read withDataSourceProviderFn(
SerializableFunction dataSourceProviderFn) {
return toBuilder().setDataSourceProviderFn(dataSourceProviderFn).build();
}
public Read withQuery(String query) {
checkArgument(query != null, "query can not be null");
return withQuery(ValueProvider.StaticValueProvider.of(query));
}
public Read withQuery(ValueProvider query) {
checkArgument(query != null, "query can not be null");
return toBuilder().setQuery(query).build();
}
public Read withStatementPreparator(StatementPreparator statementPreparator) {
checkArgumentNotNull(statementPreparator, "statementPreparator can not be null");
return toBuilder().setStatementPreparator(statementPreparator).build();
}
public Read withRowMapper(RowMapper rowMapper) {
checkArgumentNotNull(rowMapper, "rowMapper can not be null");
return toBuilder().setRowMapper(rowMapper).build();
}
/**
* @deprecated
* {@link JdbcIO} is able to infer appropriate coders from other parameters.
*/
@Deprecated
public Read withCoder(Coder coder) {
checkArgument(coder != null, "coder can not be null");
return toBuilder().setCoder(coder).build();
}
/**
* This method is used to set the size of the data that is going to be fetched and loaded in
* memory per every database call. Please refer to: {@link java.sql.Statement#setFetchSize(int)}
* It should ONLY be used if the default value throws memory errors.
*/
public Read withFetchSize(int fetchSize) {
// Note that api.java.sql.Statement#setFetchSize says it only accepts values >= 0
// and that MySQL supports using Integer.MIN_VALUE as a hint to stream the ResultSet instead
// of loading it into memory. See
// https://dev.mysql.com/doc/connector-j/8.0/en/connector-j-reference-implementation-notes.html for additional details.
checkArgument(
fetchSize >= 0 || fetchSize == Integer.MIN_VALUE,
"fetch size must be >= 0 or equal to Integer.MIN_VALUE");
return toBuilder().setFetchSize(fetchSize).build();
}
/**
* Whether to reshuffle the resulting PCollection so results are distributed to all workers. The
* default is to parallelize and should only be changed if this is known to be unnecessary.
*/
public Read withOutputParallelization(boolean outputParallelization) {
return toBuilder().setOutputParallelization(outputParallelization).build();
}
@Override
public PCollection expand(PBegin input) {
ValueProvider query = checkArgumentNotNull(getQuery(), "withQuery() is required");
RowMapper rowMapper = checkArgumentNotNull(getRowMapper(), "withRowMapper() is required");
SerializableFunction dataSourceProviderFn =
checkArgumentNotNull(
getDataSourceProviderFn(),
"withDataSourceConfiguration() or withDataSourceProviderFn() is required");
JdbcIO.ReadAll readAll =
JdbcIO.readAll()
.withDataSourceProviderFn(dataSourceProviderFn)
.withQuery(query)
.withRowMapper(rowMapper)
.withFetchSize(getFetchSize())
.withOutputParallelization(getOutputParallelization())
.withParameterSetter(
(element, preparedStatement) -> {
if (getStatementPreparator() != null) {
getStatementPreparator().setParameters(preparedStatement);
}
});
@Nullable Coder coder = getCoder();
if (coder != null) {
readAll = readAll.toBuilder().setCoder(coder).build();
}
return input.apply(Create.of((Void) null)).apply(readAll);
}
@Override
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder.add(DisplayData.item("query", getQuery()));
if (getRowMapper() != null) {
builder.add(DisplayData.item("rowMapper", getRowMapper().getClass().getName()));
}
if (getCoder() != null) {
builder.add(DisplayData.item("coder", getCoder().getClass().getName()));
}
if (getDataSourceProviderFn() instanceof HasDisplayData) {
((HasDisplayData) getDataSourceProviderFn()).populateDisplayData(builder);
}
}
}
/** Implementation of {@link #readAll}. */
@AutoValue
public abstract static class ReadAll
extends PTransform, PCollection> {
@Pure
abstract @Nullable SerializableFunction getDataSourceProviderFn();
@Pure
abstract @Nullable ValueProvider getQuery();
@Pure
abstract @Nullable PreparedStatementSetter getParameterSetter();
@Pure
abstract @Nullable RowMapper getRowMapper();
@Pure
abstract @Nullable Coder getCoder();
abstract int getFetchSize();
abstract boolean getOutputParallelization();
abstract Builder toBuilder();
@AutoValue.Builder
abstract static class Builder {
abstract Builder setDataSourceProviderFn(
SerializableFunction dataSourceProviderFn);
abstract Builder setQuery(ValueProvider query);
abstract Builder setParameterSetter(
PreparedStatementSetter parameterSetter);
abstract Builder setRowMapper(RowMapper rowMapper);
abstract Builder setCoder(Coder coder);
abstract Builder setFetchSize(int fetchSize);
abstract Builder setOutputParallelization(boolean outputParallelization);
abstract ReadAll build();
}
public ReadAll withDataSourceConfiguration(
DataSourceConfiguration config) {
return withDataSourceProviderFn(new DataSourceProviderFromDataSourceConfiguration(config));
}
public ReadAll withDataSourceProviderFn(
SerializableFunction dataSourceProviderFn) {
if (getDataSourceProviderFn() != null) {
throw new IllegalArgumentException(
"A dataSourceConfiguration or dataSourceProviderFn has "
+ "already been provided, and does not need to be provided again.");
}
return toBuilder().setDataSourceProviderFn(dataSourceProviderFn).build();
}
public ReadAll withQuery(String query) {
checkArgument(query != null, "JdbcIO.readAll().withQuery(query) called with null query");
return withQuery(ValueProvider.StaticValueProvider.of(query));
}
public ReadAll withQuery(ValueProvider query) {
checkArgument(query != null, "JdbcIO.readAll().withQuery(query) called with null query");
return toBuilder().setQuery(query).build();
}
public ReadAll withParameterSetter(
PreparedStatementSetter parameterSetter) {
checkArgumentNotNull(
parameterSetter,
"JdbcIO.readAll().withParameterSetter(parameterSetter) called "
+ "with null statementPreparator");
return toBuilder().setParameterSetter(parameterSetter).build();
}
public ReadAll withRowMapper(RowMapper rowMapper) {
checkArgument(
rowMapper != null,
"JdbcIO.readAll().withRowMapper(rowMapper) called with null rowMapper");
return toBuilder().setRowMapper(rowMapper).build();
}
/**
* @deprecated
* {@link JdbcIO} is able to infer appropriate coders from other parameters.
*/
@Deprecated
public ReadAll withCoder(Coder coder) {
checkArgument(coder != null, "JdbcIO.readAll().withCoder(coder) called with null coder");
return toBuilder().setCoder(coder).build();
}
/**
* This method is used to set the size of the data that is going to be fetched and loaded in
* memory per every database call. Please refer to: {@link java.sql.Statement#setFetchSize(int)}
* It should ONLY be used if the default value throws memory errors.
*/
public ReadAll withFetchSize(int fetchSize) {
// Note that api.java.sql.Statement#setFetchSize says it only accepts values >= 0
// and that MySQL supports using Integer.MIN_VALUE as a hint to stream the ResultSet instead
// of loading it into memory. See
// https://dev.mysql.com/doc/connector-j/8.0/en/connector-j-reference-implementation-notes.html for additional details.
checkArgument(
fetchSize >= 0 || fetchSize == Integer.MIN_VALUE,
"fetch size must be >= 0 or equal to Integer.MIN_VALUE");
return toBuilder().setFetchSize(fetchSize).build();
}
/**
* Whether to reshuffle the resulting PCollection so results are distributed to all workers. The
* default is to parallelize and should only be changed if this is known to be unnecessary.
*/
public ReadAll withOutputParallelization(boolean outputParallelization) {
return toBuilder().setOutputParallelization(outputParallelization).build();
}
private @Nullable Coder inferCoder(
CoderRegistry registry, SchemaRegistry schemaRegistry) {
if (getCoder() != null) {
return getCoder();
} else {
RowMapper rowMapper = getRowMapper();
TypeDescriptor outputType =
TypeDescriptors.extractFromTypeParameters(
rowMapper,
RowMapper.class,
new TypeVariableExtractor, OutputT>() {});
try {
return schemaRegistry.getSchemaCoder(outputType);
} catch (NoSuchSchemaException e) {
LOG.warn(
"Unable to infer a schema for type {}. Attempting to infer a coder without a schema.",
outputType);
}
try {
return registry.getCoder(outputType);
} catch (CannotProvideCoderException e) {
LOG.warn("Unable to infer a coder for type {}", outputType);
return null;
}
}
}
@Override
public PCollection expand(PCollection input) {
Coder coder =
inferCoder(
input.getPipeline().getCoderRegistry(), input.getPipeline().getSchemaRegistry());
checkStateNotNull(
coder,
"Unable to infer a coder for JdbcIO.readAll() transform. "
+ "Provide a coder via withCoder, or ensure that one can be inferred from the"
+ " provided RowMapper.");
PCollection output =
input
.apply(
ParDo.of(
new ReadFn<>(
checkStateNotNull(getDataSourceProviderFn()),
checkStateNotNull(getQuery()),
checkStateNotNull(getParameterSetter()),
checkStateNotNull(getRowMapper()),
getFetchSize())))
.setCoder(coder);
if (getOutputParallelization()) {
output = output.apply(new Reparallelize<>());
}
try {
TypeDescriptor typeDesc = coder.getEncodedTypeDescriptor();
SchemaRegistry registry = input.getPipeline().getSchemaRegistry();
Schema schema = registry.getSchema(typeDesc);
output.setSchema(
schema,
typeDesc,
registry.getToRowFunction(typeDesc),
registry.getFromRowFunction(typeDesc));
} catch (NoSuchSchemaException e) {
// ignore
}
return output;
}
@Override
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder.add(DisplayData.item("query", getQuery()));
if (getRowMapper() != null) {
builder.add(DisplayData.item("rowMapper", getRowMapper().getClass().getName()));
}
if (getCoder() != null) {
builder.add(DisplayData.item("coder", getCoder().getClass().getName()));
}
if (getDataSourceProviderFn() instanceof HasDisplayData) {
((HasDisplayData) getDataSourceProviderFn()).populateDisplayData(builder);
}
}
}
/** Implementation of {@link #readWithPartitions}. */
@AutoValue
public abstract static class ReadWithPartitions
extends PTransform> {
@Pure
abstract @Nullable SerializableFunction getDataSourceProviderFn();
@Pure
abstract @Nullable RowMapper getRowMapper();
@Pure
abstract @Nullable Coder getCoder();
@Pure
abstract @Nullable Integer getNumPartitions();
@Pure
abstract @Nullable String getPartitionColumn();
@Pure
abstract int getFetchSize();
@Pure
abstract boolean getUseBeamSchema();
@Pure
abstract @Nullable PartitionColumnT getLowerBound();
@Pure
abstract @Nullable PartitionColumnT getUpperBound();
@Pure
abstract @Nullable String getTable();
@Pure
abstract @Nullable TypeDescriptor getPartitionColumnType();
@Pure
abstract @Nullable JdbcReadWithPartitionsHelper getPartitionsHelper();
@Pure
abstract Builder toBuilder();
@AutoValue.Builder
abstract static class Builder {
abstract Builder setDataSourceProviderFn(
SerializableFunction dataSourceProviderFn);
abstract Builder setRowMapper(RowMapper rowMapper);
abstract Builder setCoder(Coder coder);
abstract Builder setNumPartitions(int numPartitions);
abstract Builder setPartitionColumn(String partitionColumn);
abstract Builder setLowerBound(PartitionColumnT lowerBound);
abstract Builder setUpperBound(PartitionColumnT upperBound);
abstract Builder setUseBeamSchema(boolean useBeamSchema);
abstract Builder setFetchSize(int fetchSize);
abstract Builder setTable(String tableName);
abstract Builder setPartitionColumnType(
TypeDescriptor partitionColumnType);
abstract Builder setPartitionsHelper(
JdbcReadWithPartitionsHelper partitionsHelper);
abstract ReadWithPartitions build();
}
public ReadWithPartitions withDataSourceConfiguration(
final DataSourceConfiguration config) {
return withDataSourceProviderFn(new DataSourceProviderFromDataSourceConfiguration(config));
}
public ReadWithPartitions withDataSourceProviderFn(
SerializableFunction dataSourceProviderFn) {
return toBuilder().setDataSourceProviderFn(dataSourceProviderFn).build();
}
public ReadWithPartitions withRowMapper(RowMapper rowMapper) {
checkNotNull(rowMapper, "rowMapper can not be null");
return toBuilder().setRowMapper(rowMapper).build();
}
/**
* @deprecated
* {@link JdbcIO} is able to infer appropriate coders from other parameters.
*/
@Deprecated
public ReadWithPartitions withCoder(Coder coder) {
checkNotNull(coder, "coder can not be null");
return toBuilder().setCoder(coder).build();
}
/**
* The number of partitions. This, along with withLowerBound and withUpperBound, form partitions
* strides for generated WHERE clause expressions used to split the column withPartitionColumn
* evenly. When the input is less than 1, the number is set to 1.
*/
public ReadWithPartitions withNumPartitions(int numPartitions) {
checkArgument(numPartitions > 0, "numPartitions can not be less than 1");
return toBuilder().setNumPartitions(numPartitions).build();
}
/** The name of a column of numeric type that will be used for partitioning. */
public ReadWithPartitions withPartitionColumn(String partitionColumn) {
checkNotNull(partitionColumn, "partitionColumn can not be null");
return toBuilder().setPartitionColumn(partitionColumn).build();
}
/** The number of rows to fetch from the database in the same {@link ResultSet} round-trip. */
public ReadWithPartitions withFetchSize(int fetchSize) {
checkArgument(fetchSize > 0, "fetchSize can not be less than 1");
return toBuilder().setFetchSize(fetchSize).build();
}
/** Data output type is {@link Row}, and schema is auto-inferred from the database. */
public ReadWithPartitions withRowOutput() {
return toBuilder().setUseBeamSchema(true).build();
}
public ReadWithPartitions withLowerBound(PartitionColumnT lowerBound) {
return toBuilder().setLowerBound(lowerBound).build();
}
public ReadWithPartitions withUpperBound(PartitionColumnT upperBound) {
return toBuilder().setUpperBound(upperBound).build();
}
/** Name of the table in the external database. Can be used to pass a user-defined subqery. */
public ReadWithPartitions withTable(String tableName) {
checkNotNull(tableName, "table can not be null");
return toBuilder().setTable(tableName).build();
}
private static final int EQUAL = 0;
@Override
public PCollection expand(PBegin input) {
SerializableFunction dataSourceProviderFn =
checkStateNotNull(
getDataSourceProviderFn(),
"withDataSourceConfiguration() or withDataSourceProviderFn() is required");
String partitionColumn =
checkStateNotNull(getPartitionColumn(), "withPartitionColumn() is required");
String table = checkStateNotNull(getTable(), "withTable() is required");
checkArgument(
// We XOR so that only one of these is true / provided. (^ is an xor operator : ))
getUseBeamSchema() ^ getRowMapper() != null,
"Provide only withRowOutput() or withRowMapper() arguments for "
+ "JdbcIO.readWithPartitions). These are mutually exclusive.");
checkArgument(
(getUpperBound() != null) == (getLowerBound() != null),
"When providing either lower or upper bound, both "
+ "parameters are mandatory for JdbcIO.readWithPartitions");
if (getLowerBound() != null
&& getUpperBound() != null
&& getLowerBound() instanceof Comparable>) {
// Not all partition types are comparable. For example, LocalDateTime, which is a valid
// partitioning type, is not Comparable, so we can't enforce this for all sorts of
// partitioning.
checkArgument(
((Comparable) getLowerBound()).compareTo(getUpperBound()) < EQUAL,
"The lower bound of partitioning column is larger or equal than the upper bound");
}
JdbcReadWithPartitionsHelper partitionsHelper = getPartitionsHelper();
if (partitionsHelper == null) {
partitionsHelper =
JdbcUtil.getPartitionsHelper(
checkStateNotNull(
getPartitionColumnType(),
"Provide partitionColumnType or partitionsHelper for JdbcIO.readWithPartitions()"));
checkNotNull(
partitionsHelper,
"readWithPartitions only supports the following types: %s",
JdbcUtil.PRESET_HELPERS.keySet());
}
PCollection>> params;
if (getLowerBound() == null && getUpperBound() == null) {
String query =
String.format(
"SELECT min(%s), max(%s) FROM %s", partitionColumn, partitionColumn, table);
if (getNumPartitions() == null) {
query =
String.format(
"SELECT min(%s), max(%s), count(*) FROM %s",
partitionColumn, partitionColumn, table);
}
params =
input
.apply(
JdbcIO.>>read()
.withQuery(query)
.withDataSourceProviderFn(dataSourceProviderFn)
.withRowMapper(checkStateNotNull(partitionsHelper))
.withFetchSize(getFetchSize()))
.apply(
MapElements.via(
new SimpleFunction<
KV>,
KV>>() {
@Override
public KV> apply(
KV> input) {
KV> result;
if (getNumPartitions() == null) {
// In this case, we use the table row count to infer a number of
// partitions.
// We take the square root of the number of rows, and divide it by 10
// to keep a relatively low number of partitions, given that an RDBMS
// cannot usually accept a very large number of connections.
long numPartitions =
Math.max(
1, Math.round(Math.floor(Math.sqrt(input.getKey()) / 10)));
result = KV.of(numPartitions, input.getValue());
} else {
result = KV.of(getNumPartitions().longValue(), input.getValue());
}
LOG.info(
"Inferred min: {} - max: {} - numPartitions: {}",
result.getValue().getKey(),
result.getValue().getValue(),
result.getKey());
return result;
}
}));
} else {
params =
input.apply(
Create.of(
KV.of(
checkStateNotNull(getNumPartitions()).longValue(),
KV.of(getLowerBound(), getUpperBound()))));
}
RowMapper rowMapper = null;
Schema schema = null;
if (getUseBeamSchema()) {
schema =
ReadRows.inferBeamSchema(
dataSourceProviderFn.apply(null), String.format("SELECT * FROM %s", getTable()));
rowMapper = (RowMapper) SchemaUtil.BeamRowMapper.of(schema);
} else {
rowMapper = getRowMapper();
}
checkStateNotNull(rowMapper);
PCollection> ranges =
params
.apply(
"Partitioning",
ParDo.of(new PartitioningFn<>(checkStateNotNull(partitionsHelper))))
.apply("Reshuffle partitions", Reshuffle.viaRandomKey());
JdbcIO.ReadAll, T> readAll =
JdbcIO., T>readAll()
.withDataSourceProviderFn(dataSourceProviderFn)
.withQuery(
String.format(
"select * from %1$s where %2$s >= ? and %2$s < ?", table, partitionColumn))
.withRowMapper(rowMapper)
.withFetchSize(getFetchSize())
.withParameterSetter(checkStateNotNull(partitionsHelper))
.withOutputParallelization(false);
if (getUseBeamSchema()) {
checkStateNotNull(schema);
readAll = readAll.withCoder((Coder) RowCoder.of(schema));
} else if (getCoder() != null) {
readAll = readAll.withCoder(getCoder());
}
return ranges.apply("Read ranges", readAll);
}
@Override
public void populateDisplayData(DisplayData.Builder builder) {
super.populateDisplayData(builder);
builder.add(
DisplayData.item(
"rowMapper",
getRowMapper() == null
? "auto-infer"
: getRowMapper().getClass().getCanonicalName()));
if (getCoder() != null) {
builder.add(DisplayData.item("coder", getCoder().getClass().getName()));
}
builder.add(DisplayData.item("partitionColumn", getPartitionColumn()));
builder.add(DisplayData.item("table", getTable()));
builder.add(
DisplayData.item(
"numPartitions",
getNumPartitions() == null ? "auto-infer" : getNumPartitions().toString()));
builder.add(
DisplayData.item(
"lowerBound", getLowerBound() == null ? "auto-infer" : getLowerBound().toString()));
builder.add(
DisplayData.item(
"upperBound", getUpperBound() == null ? "auto-infer" : getUpperBound().toString()));
if (getDataSourceProviderFn() instanceof HasDisplayData) {
((HasDisplayData) getDataSourceProviderFn()).populateDisplayData(builder);
}
}
}
/** A {@link DoFn} executing the SQL query to read from the database. */
private static class ReadFn extends DoFn {
private final SerializableFunction dataSourceProviderFn;
private final ValueProvider query;
private final PreparedStatementSetter parameterSetter;
private final RowMapper rowMapper;
private final int fetchSize;
private @Nullable DataSource dataSource;
private @Nullable Connection connection;
private ReadFn(
SerializableFunction dataSourceProviderFn,
ValueProvider query,
PreparedStatementSetter parameterSetter,
RowMapper rowMapper,
int fetchSize) {
this.dataSourceProviderFn = dataSourceProviderFn;
this.query = query;
this.parameterSetter = parameterSetter;
this.rowMapper = rowMapper;
this.fetchSize = fetchSize;
}
@Setup
public void setup() throws Exception {
dataSource = dataSourceProviderFn.apply(null);
}
private Connection getConnection() throws SQLException {
if (this.connection == null) {
this.connection = checkStateNotNull(this.dataSource).getConnection();
}
return this.connection;
}
@ProcessElement
// Spotbugs seems to not understand the nested try-with-resources
@SuppressFBWarnings({
"OBL_UNSATISFIED_OBLIGATION",
"ODR_OPEN_DATABASE_RESOURCE", // connection closed in finishbundle
})
public void processElement(ProcessContext context) throws Exception {
// Only acquire the connection if we need to perform a read.
Connection connection = getConnection();
// PostgreSQL requires autocommit to be disabled to enable cursor streaming
// see https://jdbc.postgresql.org/documentation/head/query.html#query-with-cursor
LOG.info("Autocommit has been disabled");
connection.setAutoCommit(false);
try (PreparedStatement statement =
connection.prepareStatement(
query.get(), ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY)) {
statement.setFetchSize(fetchSize);
parameterSetter.setParameters(context.element(), statement);
try (ResultSet resultSet = statement.executeQuery()) {
while (resultSet.next()) {
context.output(rowMapper.mapRow(resultSet));
}
}
}
}
@FinishBundle
public void finishBundle() throws Exception {
cleanUpConnection();
}
@Teardown
public void tearDown() throws Exception {
cleanUpConnection();
}
private void cleanUpConnection() throws Exception {
if (connection != null) {
try {
connection.close();
} finally {
connection = null;
}
}
}
}
/**
* Builder used to help with retry configuration for {@link JdbcIO}. The retry configuration
* accepts maxAttempts and maxDuration for {@link FluentBackoff}.
*/
@AutoValue
public abstract static class RetryConfiguration implements Serializable {
abstract int getMaxAttempts();
abstract @Nullable Duration getMaxDuration();
abstract @Nullable Duration getInitialDuration();
abstract RetryConfiguration.Builder builder();
@AutoValue.Builder
abstract static class Builder {
abstract Builder setMaxAttempts(int maxAttempts);
abstract Builder setMaxDuration(Duration maxDuration);
abstract Builder setInitialDuration(Duration initialDuration);
abstract RetryConfiguration build();
}
public static RetryConfiguration create(
int maxAttempts, @Nullable Duration maxDuration, @Nullable Duration initialDuration) {
if (maxDuration == null || maxDuration.equals(Duration.ZERO)) {
maxDuration = DEFAULT_MAX_CUMULATIVE_BACKOFF;
}
if (initialDuration == null || initialDuration.equals(Duration.ZERO)) {
initialDuration = DEFAULT_INITIAL_BACKOFF;
}
checkArgument(maxAttempts > 0, "maxAttempts must be greater than 0");
return new AutoValue_JdbcIO_RetryConfiguration.Builder()
.setMaxAttempts(maxAttempts)
.setInitialDuration(initialDuration)
.setMaxDuration(maxDuration)
.build();
}
}
/**
* An interface used by the JdbcIO Write to set the parameters of the {@link PreparedStatement}
* used to setParameters into the database.
*/
@FunctionalInterface
public interface PreparedStatementSetter extends Serializable {
void setParameters(T element, PreparedStatement preparedStatement) throws Exception;
}
/**
* An interface used to control if we retry the statements when a {@link SQLException} occurs. If
* {@link RetryStrategy#apply(SQLException)} returns true, {@link Write} tries to replay the
* statements.
*/
@FunctionalInterface
public interface RetryStrategy extends Serializable {
boolean apply(SQLException sqlException);
}
/**
* This class is used as the default return value of {@link JdbcIO#write()}.
*
* All methods in this class delegate to the appropriate method of {@link JdbcIO.WriteVoid}.
*/
public static class Write extends PTransform, PDone> {
WriteVoid inner;
Write() {
this(JdbcIO.writeVoid());
}
Write(WriteVoid inner) {
this.inner = inner;
}
/** See {@link WriteVoid#withAutoSharding()}. */
public Write withAutoSharding() {
return new Write<>(inner.withAutoSharding());
}
/** See {@link WriteVoid#withDataSourceConfiguration(DataSourceConfiguration)}. */
public Write withDataSourceConfiguration(DataSourceConfiguration config) {
return new Write<>(inner.withDataSourceConfiguration(config));
}
/** See {@link WriteVoid#withDataSourceProviderFn(SerializableFunction)}. */
public Write withDataSourceProviderFn(
SerializableFunction dataSourceProviderFn) {
return new Write<>(inner.withDataSourceProviderFn(dataSourceProviderFn));
}
/** See {@link WriteVoid#withStatement(String)}. */
public Write withStatement(String statement) {
return new Write<>(inner.withStatement(statement));
}
/** See {@link WriteVoid#withPreparedStatementSetter(PreparedStatementSetter)}. */
public Write withPreparedStatementSetter(PreparedStatementSetter setter) {
return new Write<>(inner.withPreparedStatementSetter(setter));
}
/** See {@link WriteVoid#withBatchSize(long)}. */
public Write withBatchSize(long batchSize) {
return new Write<>(inner.withBatchSize(batchSize));
}
/** See {@link WriteVoid#withMaxBatchBufferingDuration(long)}. */
public Write withMaxBatchBufferingDuration(long maxBatchBufferingDuration) {
return new Write<>(inner.withMaxBatchBufferingDuration(maxBatchBufferingDuration));
}
/** See {@link WriteVoid#withRetryStrategy(RetryStrategy)}. */
public Write withRetryStrategy(RetryStrategy retryStrategy) {
return new Write<>(inner.withRetryStrategy(retryStrategy));
}
/** See {@link WriteVoid#withRetryConfiguration(RetryConfiguration)}. */
public Write withRetryConfiguration(RetryConfiguration retryConfiguration) {
return new Write<>(inner.withRetryConfiguration(retryConfiguration));
}
/** See {@link WriteVoid#withTable(String)}. */
public Write withTable(String table) {
return new Write<>(inner.withTable(table));
}
/**
* Returns {@link WriteVoid} transform which can be used in {@link Wait#on(PCollection[])} to
* wait until all data is written.
*
* Example: write a {@link PCollection} to one database and then to another database, making
* sure that writing a window of data to the second database starts only after the respective
* window has been fully written to the first database.
*
*
{@code
* PCollection firstWriteResults = data.apply(JdbcIO.write()
* .withDataSourceConfiguration(CONF_DB_1).withResults());
* data.apply(Wait.on(firstWriteResults))
* .apply(JdbcIO.write().withDataSourceConfiguration(CONF_DB_2));
* }
*/
public WriteVoid withResults() {
return inner;
}
/**
* Returns {@link WriteWithResults} transform that could return a specific result.
*
* See {@link WriteWithResults}
*/
public WriteWithResults withWriteResults(
RowMapper rowMapper) {
return new AutoValue_JdbcIO_WriteWithResults.Builder()
.setRowMapper(rowMapper)
.setRetryStrategy(inner.getRetryStrategy())
.setRetryConfiguration(inner.getRetryConfiguration())
.setDataSourceProviderFn(inner.getDataSourceProviderFn())
.setPreparedStatementSetter(inner.getPreparedStatementSetter())
.setStatement(inner.getStatement())
.setTable(inner.getTable())
.setAutoSharding(inner.getAutoSharding())
.build();
}
@Override
public void populateDisplayData(DisplayData.Builder builder) {
inner.populateDisplayData(builder);
}
@Override
public PDone expand(PCollection input) {
inner.expand(input);
return PDone.in(input.getPipeline());
}
}
/* The maximum number of elements that will be included in a batch. */
static PCollection> batchElements(
PCollection input,
@Nullable Boolean withAutoSharding,
long batchSize,
long maxBatchBufferingDuration) {
PCollection> iterables;
if (input.isBounded() == IsBounded.UNBOUNDED) {
PCollection> keyedInput = input.apply(WithKeys.of(""));
GroupIntoBatches groupTransform =
GroupIntoBatches.ofSize(batchSize)
.withMaxBufferingDuration(Duration.millis(maxBatchBufferingDuration));
if (withAutoSharding != null && withAutoSharding) {
// unbounded and withAutoSharding enabled, group into batches with shardedKey
iterables = keyedInput.apply(groupTransform.withShardedKey()).apply(Values.create());
} else {
// unbounded and without auto sharding, group into batches of assigned max size
iterables = keyedInput.apply(groupTransform).apply(Values.create());
}
} else {
iterables =
input.apply(
ParDo.of(
new DoFn>() {
@Nullable List outputList;
@ProcessElement
@SuppressWarnings(
"nullness") // https://github.com/typetools/checker-framework/issues/6389
public void process(ProcessContext c) {
if (outputList == null) {
outputList = new ArrayList<>();
}
outputList.add(c.element());
if (outputList.size() > batchSize) {
c.output(outputList);
outputList = null;
}
}
@FinishBundle
public void finish(FinishBundleContext c) {
if (outputList != null && outputList.size() > 0) {
c.output(outputList, Instant.now(), GlobalWindow.INSTANCE);
}
outputList = null;
}
}));
}
return iterables;
}
/** Interface implemented by functions that sets prepared statement data. */
@FunctionalInterface
interface PreparedStatementSetCaller extends Serializable {
void set(
Row element,
PreparedStatement preparedStatement,
int prepareStatementIndex,
SchemaUtil.FieldWithIndex schemaFieldWithIndex)
throws SQLException;
}
/**
* A {@link PTransform} to write to a JDBC datasource. Executes statements one by one.
*
* The INSERT, UPDATE, and DELETE commands sometimes have an optional RETURNING clause that
* supports obtaining data from modified rows while they are being manipulated. Output {@link
* PCollection} of this transform is a collection of such returning results mapped by {@link
* RowMapper}.
*/
@AutoValue
public abstract static class WriteWithResults