|
18 | 18 | package org.apache.beam.sdk.io.jdbc; |
19 | 19 |
|
20 | 20 | import static org.junit.Assert.assertEquals; |
| 21 | +import static org.junit.Assert.assertNotNull; |
21 | 22 |
|
22 | 23 | import java.sql.Connection; |
23 | 24 | import java.sql.PreparedStatement; |
@@ -85,6 +86,100 @@ public void testPartitionedRead() { |
85 | 86 | pipeline.run(); |
86 | 87 | } |
87 | 88 |
|
| 89 | + @Test |
| 90 | + public void testPartitionedReadWithExplicitSchema() { |
| 91 | + JdbcSchemaIOProvider provider = new JdbcSchemaIOProvider(); |
| 92 | + |
| 93 | + // Define a custom schema with different field names |
| 94 | + Schema customSchema = |
| 95 | + Schema.of( |
| 96 | + Schema.Field.of("CUSTOMER_NAME", Schema.FieldType.STRING).withNullable(true), |
| 97 | + Schema.Field.of("CUSTOMER_ID", Schema.FieldType.INT32).withNullable(true)); |
| 98 | + |
| 99 | + Row config = |
| 100 | + Row.withSchema(provider.configurationSchema()) |
| 101 | + .withFieldValue("driverClassName", DATA_SOURCE_CONFIGURATION.getDriverClassName().get()) |
| 102 | + .withFieldValue("jdbcUrl", DATA_SOURCE_CONFIGURATION.getUrl().get()) |
| 103 | + .withFieldValue("username", "") |
| 104 | + .withFieldValue("password", "") |
| 105 | + .withFieldValue("partitionColumn", "id") |
| 106 | + .withFieldValue("partitions", (short) 10) |
| 107 | + .build(); |
| 108 | + |
| 109 | + JdbcSchemaIOProvider.JdbcSchemaIO schemaIO = |
| 110 | + provider.from( |
| 111 | + String.format("(select name,id from %s) as subq", READ_TABLE_NAME), |
| 112 | + config, |
| 113 | + customSchema); |
| 114 | + |
| 115 | + PCollection<Row> output = pipeline.apply(schemaIO.buildReader()); |
| 116 | + |
| 117 | + // Verify the schema is the one we provided |
| 118 | + assertEquals(customSchema, output.getSchema()); |
| 119 | + |
| 120 | + // Verify row count |
| 121 | + Long expected = Long.valueOf(EXPECTED_ROW_COUNT); |
| 122 | + PAssert.that(output.apply(Count.globally())).containsInAnyOrder(expected); |
| 123 | + |
| 124 | + // Verify field names match our custom schema |
| 125 | + PAssert.that(output) |
| 126 | + .satisfies( |
| 127 | + rows -> { |
| 128 | + for (Row row : rows) { |
| 129 | + assertNotNull(row.getString("CUSTOMER_NAME")); |
| 130 | + assertNotNull(row.getInt32("CUSTOMER_ID")); |
| 131 | + } |
| 132 | + return null; |
| 133 | + }); |
| 134 | + |
| 135 | + pipeline.run(); |
| 136 | + } |
| 137 | + |
| 138 | + @Test |
| 139 | + public void testReadWithExplicitSchema() { |
| 140 | + JdbcSchemaIOProvider provider = new JdbcSchemaIOProvider(); |
| 141 | + |
| 142 | + // Define a custom schema with different field names |
| 143 | + Schema customSchema = |
| 144 | + Schema.of( |
| 145 | + Schema.Field.of("CUSTOMER_NAME", Schema.FieldType.STRING).withNullable(true), |
| 146 | + Schema.Field.of("CUSTOMER_ID", Schema.FieldType.INT32).withNullable(true)); |
| 147 | + |
| 148 | + Row config = |
| 149 | + Row.withSchema(provider.configurationSchema()) |
| 150 | + .withFieldValue("driverClassName", DATA_SOURCE_CONFIGURATION.getDriverClassName().get()) |
| 151 | + .withFieldValue("jdbcUrl", DATA_SOURCE_CONFIGURATION.getUrl().get()) |
| 152 | + .withFieldValue("username", "") |
| 153 | + .withFieldValue("password", "") |
| 154 | + .withFieldValue("readQuery", "SELECT name, id FROM " + READ_TABLE_NAME) |
| 155 | + .build(); |
| 156 | + |
| 157 | + JdbcSchemaIOProvider.JdbcSchemaIO schemaIO = |
| 158 | + provider.from(READ_TABLE_NAME, config, customSchema); |
| 159 | + |
| 160 | + PCollection<Row> output = pipeline.apply(schemaIO.buildReader()); |
| 161 | + |
| 162 | + // Verify the schema is the one we provided |
| 163 | + assertEquals(customSchema, output.getSchema()); |
| 164 | + |
| 165 | + // Verify row count |
| 166 | + Long expected = Long.valueOf(EXPECTED_ROW_COUNT); |
| 167 | + PAssert.that(output.apply(Count.globally())).containsInAnyOrder(expected); |
| 168 | + |
| 169 | + // Verify field names match our custom schema |
| 170 | + PAssert.that(output) |
| 171 | + .satisfies( |
| 172 | + rows -> { |
| 173 | + for (Row row : rows) { |
| 174 | + assertNotNull(row.getString("CUSTOMER_NAME")); |
| 175 | + assertNotNull(row.getInt32("CUSTOMER_ID")); |
| 176 | + } |
| 177 | + return null; |
| 178 | + }); |
| 179 | + |
| 180 | + pipeline.run(); |
| 181 | + } |
| 182 | + |
88 | 183 | // This test shouldn't work because we only support numeric and datetime columns and we are trying |
89 | 184 | // to use a string column as our partition source. |
90 | 185 | @Test |
|
0 commit comments