Skip to content
Closed
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 @@ -39,6 +39,7 @@
* An implementation of {@link SchemaIOProvider} for reading and writing JSON payloads with {@link
* JdbcIO}.
*/
@SuppressWarnings({"unsafe"})
@Internal
@AutoService(SchemaIOProvider.class)
public class JdbcSchemaIOProvider implements SchemaIOProvider {
Expand Down Expand Up @@ -67,6 +68,10 @@ public Schema configurationSchema() {
.addNullableField("fetchSize", FieldType.INT16)
.addNullableField("outputParallelization", FieldType.BOOLEAN)
.addNullableField("autosharding", FieldType.BOOLEAN)
// Partitioning support. If you specify a partition column we will use that instead of
// readQuery
.addNullableField("partitionColumn", FieldType.STRING)
.addNullableField("partitions", FieldType.INT16)
.build();
}

Expand Down Expand Up @@ -110,26 +115,48 @@ public PTransform<PBegin, PCollection<Row>> buildReader() {
return new PTransform<PBegin, PCollection<Row>>() {
@Override
public PCollection<Row> expand(PBegin input) {
@Nullable String readQuery = config.getString("readQuery");
if (readQuery == null) {
readQuery = String.format("SELECT * FROM %s", location);
}

JdbcIO.ReadRows readRows =
JdbcIO.readRows()
.withDataSourceConfiguration(getDataSourceConfiguration())
.withQuery(readQuery);

@Nullable Short fetchSize = config.getInt16("fetchSize");
if (fetchSize != null) {
readRows = readRows.withFetchSize(fetchSize);
}

@Nullable Boolean outputParallelization = config.getBoolean("outputParallelization");
if (outputParallelization != null) {
readRows = readRows.withOutputParallelization(outputParallelization);
// If we define a partition column we need to go a different route
@Nullable String partitionColumn = config.getString("partitionColumn");
Copy link
Contributor

Choose a reason for hiding this comment

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

Just note that for newly introduced parameters it may need to check the existence of the field first, similar to #25062 otherwise may causing error in go SDK's xlang jdbc wrapper

Copy link
Contributor

Choose a reason for hiding this comment

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

Hi @byronellis if you do not have time, I am willing to fix the comments in order to make it in before Beam 2.46.0 cut.

Copy link
Contributor

Choose a reason for hiding this comment

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

opened #25577 fixed a couple of things and test passed there

if (partitionColumn != null) {
JdbcIO.ReadWithPartitions<?, ?> readRows =
JdbcIO.readWithPartitions()
.withDataSourceConfiguration(getDataSourceConfiguration())
.withTable(location)
.withPartitionColumn(partitionColumn)
.withRowOutput();
@Nullable Short partitions = config.getInt16("partitions");
if (partitions != null) {
readRows = readRows.withNumPartitions(partitions);
}

// Need to do a cast here
return input.apply((JdbcIO.ReadWithPartitions<Row, ?>) readRows);

} else {

@Nullable String readQuery = config.getString("readQuery");
if (readQuery == null) {
readQuery = String.format("SELECT * FROM %s", location);
}

JdbcIO.ReadRows readRows =
JdbcIO.readRows()
.withDataSourceConfiguration(getDataSourceConfiguration())
.withQuery(readQuery);

@Nullable Short fetchSize = config.getInt16("fetchSize");
if (fetchSize != null) {
readRows = readRows.withFetchSize(fetchSize);
}

@Nullable Boolean outputParallelization = config.getBoolean("outputParallelization");
if (outputParallelization != null) {
readRows = readRows.withOutputParallelization(outputParallelization);
}

return input.apply(readRows);
}
return input.apply(readRows);
}
};
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,129 @@
/*
* 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 java.sql.Connection;
import java.sql.PreparedStatement;
import java.sql.SQLException;
import javax.sql.DataSource;
import org.apache.beam.sdk.io.common.DatabaseTestHelper;
import org.apache.beam.sdk.io.common.TestRow;
import org.apache.beam.sdk.schemas.Schema;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.transforms.Count;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.Row;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

@RunWith(JUnit4.class)
public class JdbcSchemaIOProviderTest {

private static final JdbcIO.DataSourceConfiguration DATA_SOURCE_CONFIGURATION =
JdbcIO.DataSourceConfiguration.create(
"org.apache.derby.jdbc.EmbeddedDriver", "jdbc:derby:memory:testDB;create=true");
private static final int EXPECTED_ROW_COUNT = 1000;

private static final DataSource DATA_SOURCE = DATA_SOURCE_CONFIGURATION.buildDatasource();
private static final String READ_TABLE_NAME = DatabaseTestHelper.getTestTableName("UT_READ");

@Rule public final transient TestPipeline pipeline = TestPipeline.create();

@BeforeClass
public static void beforeClass() throws Exception {
// by default, derby uses a lock timeout of 60 seconds. In order to speed up the test
// and detect the lock faster, we decrease this timeout
System.setProperty("derby.locks.waitTimeout", "2");
System.setProperty("derby.stream.error.file", "build/derby.log");

DatabaseTestHelper.createTable(DATA_SOURCE, READ_TABLE_NAME);
addInitialData(DATA_SOURCE, READ_TABLE_NAME);
}

@Test
public void testPartitionedRead() {
JdbcSchemaIOProvider provider = new JdbcSchemaIOProvider();

Row config =
Row.withSchema(provider.configurationSchema())
.withFieldValue("driverClassName", DATA_SOURCE_CONFIGURATION.getDriverClassName().get())
.withFieldValue("jdbcUrl", DATA_SOURCE_CONFIGURATION.getUrl().get())
.withFieldValue("username", "")
.withFieldValue("password", "")
.withFieldValue("partitionColumn", "id")
.withFieldValue("partitions", (short) 10)
.build();
JdbcSchemaIOProvider.JdbcSchemaIO schemaIO =
provider.from(READ_TABLE_NAME, config, Schema.builder().build());
PCollection<Row> output = pipeline.apply(schemaIO.buildReader());
Long expected = Long.valueOf(EXPECTED_ROW_COUNT);
PAssert.that(output.apply(Count.globally())).containsInAnyOrder(expected);
pipeline.run();
}

// This test shouldn't work because we only support numeric and datetime columns and we are trying
// to use a string
// column as our partition source
@Test
public void testPartitionedReadThatShouldntWork() throws Exception {
JdbcSchemaIOProvider provider = new JdbcSchemaIOProvider();

Row config =
Row.withSchema(provider.configurationSchema())
.withFieldValue("driverClassName", DATA_SOURCE_CONFIGURATION.getDriverClassName().get())
.withFieldValue("jdbcUrl", DATA_SOURCE_CONFIGURATION.getUrl().get())
.withFieldValue("username", "")
.withFieldValue("password", "")
.withFieldValue("partitionColumn", "name")
.withFieldValue("partitions", (short) 10)
.build();
JdbcSchemaIOProvider.JdbcSchemaIO schemaIO =
provider.from(READ_TABLE_NAME, config, Schema.builder().build());
PCollection<Row> output = pipeline.apply(schemaIO.buildReader());
Long expected = Long.valueOf(EXPECTED_ROW_COUNT);
PAssert.that(output.apply(Count.globally())).containsInAnyOrder(expected);
try {
pipeline.run();
} catch (Exception e) {
e.printStackTrace();
return;
}
throw new Exception("Did not throw an exception");
}

/** Create test data that is consistent with that generated by TestRow. */
private static void addInitialData(DataSource dataSource, String tableName) throws SQLException {
try (Connection connection = dataSource.getConnection()) {
connection.setAutoCommit(false);
try (PreparedStatement preparedStatement =
connection.prepareStatement(String.format("insert into %s values (?,?)", tableName))) {
for (int i = 0; i < EXPECTED_ROW_COUNT; i++) {
preparedStatement.clearParameters();
preparedStatement.setInt(1, i);
preparedStatement.setString(2, TestRow.getNameForSeed(i));
preparedStatement.executeUpdate();
}
}
connection.commit();
}
}
}
17 changes: 17 additions & 0 deletions sdks/python/apache_beam/io/external/xlang_jdbcio_it_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -201,6 +201,23 @@ def test_xlang_jdbc_write_read(self, database):

assert_that(result, equal_to(expected_row))

# Try the same read using the partitioned reader code path. Outputs should be the same.
with TestPipeline() as p:
p.not_use_test_runner_api = True
result = (
p
| 'Partitioned read from jdbc' >> ReadFromJdbc(
table_name=table_name,
partition_column='f_id',
partitions=3,
driver_class_name=self.driver_class_name,
jdbc_url=self.jdbc_url,
username=self.username,
password=self.password,
classpath=classpath))

assert_that(result, equal_to(expected_row))

# Creating a container with testcontainers sometimes raises ReadTimeout
# error. In java there are 2 retries set by default.
def start_db_container(self, retries, container_init):
Expand Down
9 changes: 9 additions & 0 deletions sdks/python/apache_beam/io/jdbc.py
Original file line number Diff line number Diff line change
Expand Up @@ -125,6 +125,9 @@ def default_io_expansion_service(classpath=None):
('fetch_size', typing.Optional[int]),
('output_parallelization', typing.Optional[bool]),
('autosharding', typing.Optional[bool]),
('partitionColumn', typing.Optional[str]),
('tableName', typing.Optional[str]),
Copy link
Contributor

Choose a reason for hiding this comment

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

Was the tableName configuration field meant to be added?

('partitions', typing.Optional[bool])
],
)

Expand Down Expand Up @@ -273,6 +276,8 @@ def __init__(
query=None,
output_parallelization=None,
fetch_size=None,
partition_column=None,
partitions=None,
connection_properties=None,
connection_init_sqls=None,
expansion_service=None,
Expand All @@ -288,6 +293,8 @@ def __init__(
:param query: sql query to be executed
:param output_parallelization: is output parallelization on
:param fetch_size: how many rows to fetch
:param partition_column: enable partitioned reads by splitting on this column.
:param partitions: override the default number of splits when using partition_column
:param connection_properties: properties of the jdbc connection
passed as string with format
[propertyName=property;]*
Expand Down Expand Up @@ -324,6 +331,8 @@ def __init__(
fetch_size=fetch_size,
output_parallelization=output_parallelization,
autosharding=None,
partition_column=partition_column,
partitions=partitions
))),
),
expansion_service or default_io_expansion_service(classpath),
Expand Down