Skip to content
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 @@ -214,7 +214,7 @@ protected JdbcIO.DataSourceConfiguration dataSourceConfiguration() {

@Override
public PCollectionRowTuple expand(PCollectionRowTuple input) {
config.validate();
config.validate(jdbcType);
// If we define a partition column, we follow a different route.
@Nullable String partitionColumn = config.getPartitionColumn();
@Nullable String location = config.getLocation();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThrows;

import com.google.auto.service.AutoService;
import java.sql.Connection;
import java.sql.PreparedStatement;
import java.sql.SQLException;
Expand All @@ -47,6 +48,25 @@
@RunWith(JUnit4.class)
public class JdbcReadSchemaTransformProviderTest {

@AutoService(SchemaTransformProvider.class)
public static class ReadFromDerbySchemaTransformProvider extends JdbcReadSchemaTransformProvider {

@Override
public String identifier() {
return "beam:schematransform:org.apache.beam:derby_read:v1";
}

@Override
public String description() {
return inheritedDescription("Derby", "ReadFromDerby", "derby", 5432);
}

@Override
protected String jdbcType() {
return "derby";
}
}

private static final JdbcIO.DataSourceConfiguration DATA_SOURCE_CONFIGURATION =
JdbcIO.DataSourceConfiguration.create(
"org.apache.derby.jdbc.EmbeddedDriver", "jdbc:derby:memory:testDB;create=true");
Expand Down Expand Up @@ -199,6 +219,32 @@ public void testReadWithJdbcTypeSpecified() {
pipeline.run();
}

@Test
public void testReadWithJdbcDerbyTransformTypeSpecified() {
JdbcReadSchemaTransformProvider provider = null;
for (SchemaTransformProvider p : ServiceLoader.load(SchemaTransformProvider.class)) {
if (p instanceof ReadFromDerbySchemaTransformProvider) {
provider = (JdbcReadSchemaTransformProvider) p;
break;
}
}
assertNotNull(provider);

PCollection<Row> output =
PCollectionRowTuple.empty(pipeline)
.apply(
provider.from(
ReadFromDerbySchemaTransformProvider.JdbcReadSchemaTransformConfiguration
.builder()
.setJdbcUrl(DATA_SOURCE_CONFIGURATION.getUrl().get())
.setLocation(READ_TABLE_NAME)
.build()))
.get("output");
Long expected = Long.valueOf(EXPECTED_ROW_COUNT);
PAssert.that(output.apply(Count.globally())).containsInAnyOrder(expected);
pipeline.run();
}

@Test
public void testReadWithPartitions() {
JdbcReadSchemaTransformProvider provider = null;
Expand Down
Loading