-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Add partitioned reads to JDBC SchemaIO #25240
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
Closed
Closed
Changes from all commits
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
18c87d5
Fix up the JDBCSchemaIO to support partitioned reads on a column (I b…
b1e8bc7
Merge branch 'apache:master' into beam25011-jdbc-java
byronellis f26a314
Removed the PartitionedReadSchemaTransformProvider pending further di…
1e8ebd1
Merge remote-tracking branch 'origin/beam25011-jdbc-java' into beam25…
10e5483
Merge branch 'apache:master' into beam25011-jdbc-java
byronellis File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
129 changes: 129 additions & 0 deletions
129
sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcSchemaIOProviderTest.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| 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(); | ||
| } | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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]), | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Was the |
||
| ('partitions', typing.Optional[bool]) | ||
| ], | ||
| ) | ||
|
|
||
|
|
@@ -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, | ||
|
|
@@ -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;]* | ||
|
|
@@ -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), | ||
|
|
||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
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
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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